Prechádzať zdrojové kódy

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

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

Contributed by Steve Loughran
Steve Loughran 2 rokov pred
rodič
commit
36bbde2fda
52 zmenil súbory, kde vykonal 1566 pridanie a 1279 odobranie
  1. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockCache.java
  2. 57 55
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockData.java
  3. 16 11
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManager.java
  4. 46 41
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockOperations.java
  5. 44 31
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BoundedResourcePool.java
  6. 40 16
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferData.java
  7. 81 61
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferPool.java
  8. 99 82
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java
  9. 80 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/EmptyPrefetchingStatistics.java
  10. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ExecutorServiceFuturePool.java
  11. 74 55
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/FilePosition.java
  12. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/PrefetchingStatistics.java
  13. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ResourcePool.java
  14. 14 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Retryer.java
  15. 39 32
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java
  16. 43 49
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Validate.java
  17. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/package-info.java
  18. 7 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/ExceptionAsserts.java
  19. 29 15
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/SampleDataForTests.java
  20. 7 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockCache.java
  21. 16 15
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockData.java
  22. 18 17
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockOperations.java
  23. 9 10
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBoundedResourcePool.java
  24. 13 18
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java
  25. 28 33
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java
  26. 16 10
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestExecutorServiceFuturePool.java
  27. 24 45
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestFilePosition.java
  28. 11 12
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestRetryer.java
  29. 94 75
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestValidate.java
  30. 3 3
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  31. 1 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
  32. 13 12
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ABlockManager.java
  33. 16 13
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingBlockManager.java
  34. 53 46
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java
  35. 18 14
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java
  36. 53 36
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java
  37. 99 83
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java
  38. 36 26
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java
  39. 44 38
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java
  40. 2 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/package-info.java
  41. 1 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java
  42. 0 76
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/EmptyPrefetchingStatistics.java
  43. 10 10
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java
  44. 1 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java
  45. 14 12
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java
  46. 77 54
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java
  47. 20 19
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java
  48. 81 57
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java
  49. 62 34
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java
  50. 26 15
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java
  51. 21 15
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java
  52. 4 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockCache.java → 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;

+ 57 - 55
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockData.java → 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);
   }
 }

+ 16 - 11
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockManager.java → 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.
   }

+ 46 - 41
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockOperations.java → 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<String, Kind> shortNameToKind = new HashMap<>();
 
     public static Kind fromShortName(String shortName) {
-      if (shortNameToKind.size() == 0) {
+      if (shortNameToKind.isEmpty()) {
         for (Kind kind : Kind.values()) {
           shortNameToKind.put(kind.shortName, kind);
         }
@@ -89,30 +94,30 @@ public class BlockOperations {
     }
 
     public Kind getKind() {
-      return this.kind;
+      return kind;
     }
 
     public int getBlockNumber() {
-      return this.blockNumber;
+      return blockNumber;
     }
 
     public long getTimestamp() {
-      return this.timestamp;
+      return timestamp;
     }
 
     public void getSummary(StringBuilder sb) {
-      if (this.kind.hasBlock) {
-        sb.append(String.format("%s(%d)", this.kind.shortName, this.blockNumber));
+      if (kind.hasBlock) {
+        sb.append(String.format("%s(%d)", kind.shortName, blockNumber));
       } else {
-        sb.append(String.format("%s", this.kind.shortName));
+        sb.append(String.format("%s", kind.shortName));
       }
     }
 
     public String getDebugInfo() {
-      if (this.kind.hasBlock) {
-        return String.format("--- %s(%d)", this.kind.name, this.blockNumber);
+      if (kind.hasBlock) {
+        return String.format("--- %s(%d)", kind.name, blockNumber);
       } else {
-        return String.format("... %s()", this.kind.name);
+        return String.format("... %s()", kind.name);
       }
     }
   }
@@ -137,7 +142,7 @@ public class BlockOperations {
     }
 
     public double duration() {
-      return (this.getTimestamp() - this.op.getTimestamp()) / 1e9;
+      return (getTimestamp() - op.getTimestamp()) / 1e9;
     }
   }
 
@@ -149,11 +154,11 @@ public class BlockOperations {
   }
 
   public synchronized void setDebug(boolean state) {
-    this.debugMode = state;
+    debugMode = state;
   }
 
   private synchronized Operation add(Operation op) {
-    if (this.debugMode) {
+    if (debugMode) {
       LOG.info(op.getDebugInfo());
     }
     ops.add(op);
@@ -161,63 +166,63 @@ public class BlockOperations {
   }
 
   public Operation getPrefetched(int blockNumber) {
-    Validate.checkNotNegative(blockNumber, "blockNumber");
+    checkNotNegative(blockNumber, "blockNumber");
 
-    return this.add(new Operation(Kind.GET_PREFETCHED, blockNumber));
+    return add(new Operation(Kind.GET_PREFETCHED, blockNumber));
   }
 
   public Operation getCached(int blockNumber) {
-    Validate.checkNotNegative(blockNumber, "blockNumber");
+    checkNotNegative(blockNumber, "blockNumber");
 
-    return this.add(new Operation(Kind.GET_CACHED, blockNumber));
+    return add(new Operation(Kind.GET_CACHED, blockNumber));
   }
 
   public Operation getRead(int blockNumber) {
-    Validate.checkNotNegative(blockNumber, "blockNumber");
+    checkNotNegative(blockNumber, "blockNumber");
 
-    return this.add(new Operation(Kind.GET_READ, blockNumber));
+    return add(new Operation(Kind.GET_READ, blockNumber));
   }
 
   public Operation release(int blockNumber) {
-    Validate.checkNotNegative(blockNumber, "blockNumber");
+    checkNotNegative(blockNumber, "blockNumber");
 
-    return this.add(new Operation(Kind.RELEASE, blockNumber));
+    return add(new Operation(Kind.RELEASE, blockNumber));
   }
 
   public Operation requestPrefetch(int blockNumber) {
-    Validate.checkNotNegative(blockNumber, "blockNumber");
+    checkNotNegative(blockNumber, "blockNumber");
 
-    return this.add(new Operation(Kind.REQUEST_PREFETCH, blockNumber));
+    return add(new Operation(Kind.REQUEST_PREFETCH, blockNumber));
   }
 
   public Operation prefetch(int blockNumber) {
-    Validate.checkNotNegative(blockNumber, "blockNumber");
+    checkNotNegative(blockNumber, "blockNumber");
 
-    return this.add(new Operation(Kind.PREFETCH, blockNumber));
+    return add(new Operation(Kind.PREFETCH, blockNumber));
   }
 
   public Operation cancelPrefetches() {
-    return this.add(new Operation(Kind.CANCEL_PREFETCHES, -1));
+    return add(new Operation(Kind.CANCEL_PREFETCHES, -1));
   }
 
   public Operation close() {
-    return this.add(new Operation(Kind.CLOSE, -1));
+    return add(new Operation(Kind.CLOSE, -1));
   }
 
   public Operation requestCaching(int blockNumber) {
-    Validate.checkNotNegative(blockNumber, "blockNumber");
+    checkNotNegative(blockNumber, "blockNumber");
 
-    return this.add(new Operation(Kind.REQUEST_CACHING, blockNumber));
+    return add(new Operation(Kind.REQUEST_CACHING, blockNumber));
   }
 
   public Operation addToCache(int blockNumber) {
-    Validate.checkNotNegative(blockNumber, "blockNumber");
+    checkNotNegative(blockNumber, "blockNumber");
 
-    return this.add(new Operation(Kind.CACHE_PUT, blockNumber));
+    return add(new Operation(Kind.CACHE_PUT, blockNumber));
   }
 
   public Operation end(Operation op) {
-    return this.add(new End(op));
+    return add(new End(op));
   }
 
   private static void append(StringBuilder sb, String format, Object... args) {
@@ -226,7 +231,7 @@ public class BlockOperations {
 
   public synchronized String getSummary(boolean showDebugInfo) {
     StringBuilder sb = new StringBuilder();
-    for (Operation op : this.ops) {
+    for (Operation op : ops) {
       if (op != null) {
         if (showDebugInfo) {
           sb.append(op.getDebugInfo());
@@ -239,14 +244,14 @@ public class BlockOperations {
     }
 
     sb.append("\n");
-    this.getDurationInfo(sb);
+    getDurationInfo(sb);
 
     return sb.toString();
   }
 
   public synchronized void getDurationInfo(StringBuilder sb) {
     Map<Kind, DoubleSummaryStatistics> durations = new HashMap<>();
-    for (Operation op : this.ops) {
+    for (Operation op : ops) {
       if (op instanceof End) {
         End endOp = (End) op;
         DoubleSummaryStatistics stats = durations.get(endOp.getKind());
@@ -293,7 +298,7 @@ public class BlockOperations {
     Map<Integer, List<Operation>> blockOps = new HashMap<>();
 
     // Group-by block number.
-    for (Operation op : this.ops) {
+    for (Operation op : ops) {
       if (op.blockNumber < 0) {
         continue;
       }
@@ -352,11 +357,11 @@ public class BlockOperations {
       }
     }
 
-    if (prefetchedNotUsed.size() > 0) {
+    if (!prefetchedNotUsed.isEmpty()) {
       append(sb, "Prefetched but not used: %s\n", getIntList(prefetchedNotUsed));
     }
 
-    if (cachedNotUsed.size() > 0) {
+    if (!cachedNotUsed.isEmpty()) {
       append(sb, "Cached but not used: %s\n", getIntList(cachedNotUsed));
     }
   }
@@ -410,7 +415,7 @@ public class BlockOperations {
         }
 
         if (op == null) {
-          LOG.warn("Start op not found: %s(%d)", endKind, blockNumber);
+          LOG.warn("Start op not found: {}({})", endKind, blockNumber);
         }
       }
     }

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

+ 40 - 16
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferData.java → 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<Void> action;
 
-  // Checksum of the buffer contents once in READY state.
+  /**
+   * Checksum of the buffer contents once in READY state.
+   */
   private long checksum = 0;
 
   /**
@@ -210,7 +233,6 @@ public class BufferData {
   /**
    * Updates the current state to the specified value.
    * Asserts that the current state is as expected.
-   *
    * @param newState the state to transition to.
    * @param expectedCurrentState the collection of states from which
    *        transition to {@code newState} is allowed.
@@ -218,7 +240,8 @@ public class BufferData {
    * @throws IllegalArgumentException if newState is null.
    * @throws IllegalArgumentException if expectedCurrentState is null.
    */
-  public synchronized void updateState(State newState, State... expectedCurrentState) {
+  public synchronized void updateState(State newState,
+      State... expectedCurrentState) {
     Validate.checkNotNull(newState, "newState");
     Validate.checkNotNull(expectedCurrentState, "expectedCurrentState");
 
@@ -246,7 +269,8 @@ public class BufferData {
     }
 
     String message = String.format(
-        "Expected buffer state to be '%s' but found: %s", String.join(" or ", statesStr), this);
+        "Expected buffer state to be '%s' but found: %s",
+        String.join(" or ", statesStr), this);
     throw new IllegalStateException(message);
   }
 

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

+ 99 - 82
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/CachingBlockManager.java → 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<Void> prefetchFuture = this.futurePool.executeFunction(prefetchTask);
+      Future<Void> prefetchFuture = futurePool.executeFunction(prefetchTask);
       data.setPrefetch(prefetchFuture);
-      this.ops.end(op);
+      ops.end(op);
     }
   }
 
@@ -271,21 +288,21 @@ public abstract class CachingBlockManager extends BlockManager {
    */
   @Override
   public void cancelPrefetches() {
-    BlockOperations.Operation op = this.ops.cancelPrefetches();
+    BlockOperations.Operation op = ops.cancelPrefetches();
 
-    for (BufferData data : this.bufferPool.getAll()) {
+    for (BufferData data : bufferPool.getAll()) {
       // We add blocks being prefetched to the local cache so that the prefetch is not wasted.
       if (data.stateEqualsOneOf(BufferData.State.PREFETCHING, BufferData.State.READY)) {
-        this.requestCaching(data);
+        requestCaching(data);
       }
     }
 
-    this.ops.end(op);
+    ops.end(op);
   }
 
   private void read(BufferData data) throws IOException {
     synchronized (data) {
-      this.readBlock(data, false, BufferData.State.BLANK);
+      readBlock(data, false, BufferData.State.BLANK);
     }
   }
 
@@ -293,7 +310,7 @@ public abstract class CachingBlockManager extends BlockManager {
     synchronized (data) {
       prefetchingStatistics.executorAcquired(
           Duration.between(taskQueuedStartTime, Instant.now()));
-      this.readBlock(
+      readBlock(
           data,
           true,
           BufferData.State.PREFETCHING,
@@ -304,7 +321,7 @@ public abstract class CachingBlockManager extends BlockManager {
   private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... expectedState)
       throws IOException {
 
-    if (this.closed) {
+    if (closed) {
       return;
     }
 
@@ -323,25 +340,25 @@ public abstract class CachingBlockManager extends BlockManager {
         int blockNumber = data.getBlockNumber();
 
         // Prefer reading from cache over reading from network.
-        if (this.cache.containsBlock(blockNumber)) {
-          op = this.ops.getCached(blockNumber);
-          this.cache.get(blockNumber, data.getBuffer());
+        if (cache.containsBlock(blockNumber)) {
+          op = ops.getCached(blockNumber);
+          cache.get(blockNumber, data.getBuffer());
           data.setReady(expectedState);
           return;
         }
 
         if (isPrefetch) {
           tracker = prefetchingStatistics.prefetchOperationStarted();
-          op = this.ops.prefetch(data.getBlockNumber());
+          op = ops.prefetch(data.getBlockNumber());
         } else {
-          op = this.ops.getRead(data.getBlockNumber());
+          op = ops.getRead(data.getBlockNumber());
         }
 
-        long offset = this.getBlockData().getStartOffset(data.getBlockNumber());
-        int size = this.getBlockData().getSize(data.getBlockNumber());
+        long offset = getBlockData().getStartOffset(data.getBlockNumber());
+        int size = getBlockData().getSize(data.getBlockNumber());
         ByteBuffer buffer = data.getBuffer();
         buffer.clear();
-        this.read(buffer, offset, size);
+        read(buffer, offset, size);
         buffer.flip();
         data.setReady(expectedState);
       } catch (Exception e) {
@@ -352,12 +369,12 @@ public abstract class CachingBlockManager extends BlockManager {
           tracker.failed();
         }
 
-        this.numReadErrors.incrementAndGet();
+        numReadErrors.incrementAndGet();
         data.setDone();
         throw e;
       } finally {
         if (op != null) {
-          this.ops.end(op);
+          ops.end(op);
         }
 
         if (isPrefetch) {
@@ -387,7 +404,7 @@ public abstract class CachingBlockManager extends BlockManager {
     @Override
     public Void get() {
       try {
-        this.blockManager.prefetch(data, taskQueuedStartTime);
+        blockManager.prefetch(data, taskQueuedStartTime);
       } catch (Exception e) {
         LOG.error("error during prefetch", e);
       }
@@ -409,11 +426,11 @@ public abstract class CachingBlockManager extends BlockManager {
    */
   @Override
   public void requestCaching(BufferData data) {
-    if (this.closed) {
+    if (closed) {
       return;
     }
 
-    if (this.cachingDisabled.get()) {
+    if (cachingDisabled.get()) {
       data.setDone();
       return;
     }
@@ -431,14 +448,14 @@ public abstract class CachingBlockManager extends BlockManager {
         return;
       }
 
-      if (this.cache.containsBlock(data.getBlockNumber())) {
+      if (cache.containsBlock(data.getBlockNumber())) {
         data.setDone();
         return;
       }
 
       BufferData.State state = data.getState();
 
-      BlockOperations.Operation op = this.ops.requestCaching(data.getBlockNumber());
+      BlockOperations.Operation op = ops.requestCaching(data.getBlockNumber());
       Future<Void> blockFuture;
       if (state == BufferData.State.PREFETCHING) {
         blockFuture = data.getActionFuture();
@@ -449,9 +466,9 @@ public abstract class CachingBlockManager extends BlockManager {
       }
 
       CachePutTask task = new CachePutTask(data, blockFuture, this, Instant.now());
-      Future<Void> actionFuture = this.futurePool.executeFunction(task);
+      Future<Void> actionFuture = futurePool.executeFunction(task);
       data.setCaching(actionFuture);
-      this.ops.end(op);
+      ops.end(op);
     }
   }
 
@@ -460,11 +477,11 @@ public abstract class CachingBlockManager extends BlockManager {
     prefetchingStatistics.executorAcquired(
         Duration.between(taskQueuedStartTime, Instant.now()));
 
-    if (this.closed) {
+    if (closed) {
       return;
     }
 
-    if (this.cachingDisabled.get()) {
+    if (cachingDisabled.get()) {
       data.setDone();
       return;
     }
@@ -481,7 +498,7 @@ public abstract class CachingBlockManager extends BlockManager {
       return;
     }
 
-    if (this.cachingDisabled.get()) {
+    if (cachingDisabled.get()) {
       data.setDone();
       return;
     }
@@ -494,27 +511,27 @@ public abstract class CachingBlockManager extends BlockManager {
           return;
         }
 
-        if (this.cache.containsBlock(data.getBlockNumber())) {
+        if (cache.containsBlock(data.getBlockNumber())) {
           data.setDone();
           return;
         }
 
-        op = this.ops.addToCache(data.getBlockNumber());
+        op = ops.addToCache(data.getBlockNumber());
         ByteBuffer buffer = data.getBuffer().duplicate();
         buffer.rewind();
-        this.cachePut(data.getBlockNumber(), buffer);
+        cachePut(data.getBlockNumber(), buffer);
         data.setDone();
       } catch (Exception e) {
-        this.numCachingErrors.incrementAndGet();
+        numCachingErrors.incrementAndGet();
         String message = String.format("error adding block to cache after wait: %s", data);
         LOG.error(message, e);
         data.setDone();
       }
 
       if (op != null) {
-        BlockOperations.End endOp = (BlockOperations.End) this.ops.end(op);
+        BlockOperations.End endOp = (BlockOperations.End) ops.end(op);
         if (endOp.duration() > SLOW_CACHING_THRESHOLD) {
-          if (!this.cachingDisabled.getAndSet(true)) {
+          if (!cachingDisabled.getAndSet(true)) {
             String message = String.format(
                 "Caching disabled because of slow operation (%.1f sec)", endOp.duration());
             LOG.warn(message);
@@ -529,11 +546,11 @@ public abstract class CachingBlockManager extends BlockManager {
   }
 
   protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException {
-    if (this.closed) {
+    if (closed) {
       return;
     }
 
-    this.cache.put(blockNumber, buffer);
+    cache.put(blockNumber, buffer);
   }
 
   private static class CachePutTask implements Supplier<Void> {
@@ -560,7 +577,7 @@ public abstract class CachingBlockManager extends BlockManager {
 
     @Override
     public Void get() {
-      this.blockManager.addToCacheAndRelease(this.data, this.blockFuture, taskQueuedStartTime);
+      blockManager.addToCacheAndRelease(data, blockFuture, taskQueuedStartTime);
       return null;
     }
   }
@@ -571,7 +588,7 @@ public abstract class CachingBlockManager extends BlockManager {
    * @return the number of available buffers.
    */
   public int numAvailable() {
-    return this.bufferPool.numAvailable();
+    return bufferPool.numAvailable();
   }
 
   /**
@@ -580,7 +597,7 @@ public abstract class CachingBlockManager extends BlockManager {
    * @return the number of cached buffers.
    */
   public int numCached() {
-    return this.cache.size();
+    return cache.size();
   }
 
   /**
@@ -589,7 +606,7 @@ public abstract class CachingBlockManager extends BlockManager {
    * @return the number of errors encountered when caching.
    */
   public int numCachingErrors() {
-    return this.numCachingErrors.get();
+    return numCachingErrors.get();
   }
 
   /**
@@ -598,11 +615,11 @@ public abstract class CachingBlockManager extends BlockManager {
    * @return the number of errors encountered when reading.
    */
   public int numReadErrors() {
-    return this.numReadErrors.get();
+    return numReadErrors.get();
   }
 
   BufferData getData(int blockNumber) {
-    return this.bufferPool.tryAcquire(blockNumber);
+    return bufferPool.tryAcquire(blockNumber);
   }
 
   @Override
@@ -610,11 +627,11 @@ public abstract class CachingBlockManager extends BlockManager {
     StringBuilder sb = new StringBuilder();
 
     sb.append("cache(");
-    sb.append(this.cache.toString());
+    sb.append(cache.toString());
     sb.append("); ");
 
     sb.append("pool: ");
-    sb.append(this.bufferPool.toString());
+    sb.append(bufferPool.toString());
 
     return sb.toString();
   }

+ 80 - 0
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) {
+
+  }
+}
+

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ExecutorServiceFuturePool.java → 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;

+ 74 - 55
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/FilePosition.java → 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");
   }
 }

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/PrefetchingStatistics.java → 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;
 

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ResourcePool.java → 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;
 

+ 14 - 8
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Retryer.java → 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;

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

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

+ 2 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/package-info.java → 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;

+ 7 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/ExceptionAsserts.java → 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 <E extends Exception> void assertThrows(
       Class<E> expectedExceptionClass,
       LambdaTestUtils.VoidCallable code) throws Exception {
-    assertThrows(expectedExceptionClass, null, code);
+
+    intercept(expectedExceptionClass, code);
+
   }
 }

+ 29 - 15
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/SampleDataForTests.java → 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<Object> NULL_LIST = null;
 
-  public static final List<Object> NULL_LIST  = null;
   public static final List<Object> EMPTY_LIST = new ArrayList<Object>();
+
   public static final List<Object> VALID_LIST = Arrays.asList(new Object[1]);
 }

+ 7 - 8
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockCache.java → 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++) {

+ 16 - 15
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockData.java → 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) {

+ 18 - 17
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockOperations.java → 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);

+ 9 - 10
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBoundedResourcePool.java → 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<ByteBuffer> {
+
     BufferPool(int size) {
       super(size);
     }
@@ -54,25 +56,22 @@ public class TestBoundedResourcePool extends AbstractHadoopTestBase {
     BufferPool pool = new BufferPool(5);
 
     // Verify it throws correctly.
-    ExceptionAsserts.assertThrows(
-        IllegalArgumentException.class,
+
+    intercept(IllegalArgumentException.class,
         "'size' must be a positive integer",
         () -> new BufferPool(-1));
 
-    ExceptionAsserts.assertThrows(
-        IllegalArgumentException.class,
+    intercept(IllegalArgumentException.class,
         "'size' must be a positive integer",
         () -> new BufferPool(0));
 
-    ExceptionAsserts.assertThrows(
-        IllegalArgumentException.class,
-        "'item' must not be null",
+    intercept(IllegalArgumentException.class, "'item' must not be null",
         () -> pool.release(null));
 
-    ExceptionAsserts.assertThrows(
-        IllegalArgumentException.class,
+    intercept(IllegalArgumentException.class,
         "This item is not a part of this pool",
         () -> pool.release(ByteBuffer.allocate(4)));
+
   }
 
   @Test

+ 13 - 18
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferData.java → 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;
   }
 

+ 28 - 33
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferPool.java → 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);
     }

+ 16 - 10
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java → 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<Void> future = futurePool.executeRunnable(() -> atomicBoolean.set(true));
+    Future<Void> future =
+        futurePool.executeRunnable(() -> atomicBoolean.set(true));
     future.get(30, TimeUnit.SECONDS);
     assertTrue("atomicBoolean set to true?", atomicBoolean.get());
   }
 
   @Test
   public void testSupplierSucceeds() throws Exception {
-    ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService);
+    ExecutorServiceFuturePool futurePool =
+        new ExecutorServiceFuturePool(executorService);
     final AtomicBoolean atomicBoolean = new AtomicBoolean(false);
     Future<Void> future = futurePool.executeFunction(() -> {
       atomicBoolean.set(true);
@@ -74,19 +76,23 @@ public class TestExecutorServiceFuturePool extends AbstractHadoopTestBase {
 
   @Test
   public void testRunnableFails() throws Exception {
-    ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService);
+    ExecutorServiceFuturePool futurePool =
+        new ExecutorServiceFuturePool(executorService);
     Future<Void> future = futurePool.executeRunnable(() -> {
       throw new IllegalStateException("deliberate");
     });
-    LambdaTestUtils.intercept(ExecutionException.class, () -> future.get(30, TimeUnit.SECONDS));
+    interceptFuture(IllegalStateException.class, "deliberate", 30,
+        TimeUnit.SECONDS, future);
   }
 
   @Test
   public void testSupplierFails() throws Exception {
-    ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService);
+    ExecutorServiceFuturePool futurePool =
+        new ExecutorServiceFuturePool(executorService);
     Future<Void> future = futurePool.executeFunction(() -> {
       throw new IllegalStateException("deliberate");
     });
-    LambdaTestUtils.intercept(ExecutionException.class, () -> future.get(30, TimeUnit.SECONDS));
+    interceptFuture(IllegalStateException.class, "deliberate", 30,
+        TimeUnit.SECONDS, future);
   }
 }

+ 24 - 45
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestFilePosition.java → 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

+ 11 - 12
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestRetryer.java → 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) {

+ 94 - 75
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestValidate.java → 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("<not-found>");
 
     // Should not throw.
@@ -299,24 +323,19 @@ public class TestValidate extends AbstractHadoopTestBase {
     Validate.checkPathExists(tempDir, "tempDir");
 
     // Verify it throws.
-    ExceptionAsserts.assertThrows(
-        IllegalArgumentException.class,
-        "'nullArg' must not be null",
+
+    intercept(IllegalArgumentException.class, "'nullArg' must not be null",
         () -> Validate.checkPathExists(null, "nullArg"));
 
-    ExceptionAsserts.assertThrows(
-        IllegalArgumentException.class,
+    intercept(IllegalArgumentException.class,
         "Path notFound (<not-found>) does not exist",
         () -> Validate.checkPathExists(notFound, "notFound"));
 
-    ExceptionAsserts.assertThrows(
-        IllegalArgumentException.class,
-        "must point to a directory",
+    intercept(IllegalArgumentException.class, "must point to a directory",
         () -> Validate.checkPathExistsAsDir(tempFile, "tempFile"));
 
-    ExceptionAsserts.assertThrows(
-        IllegalArgumentException.class,
-        "must point to a file",
+    intercept(IllegalArgumentException.class, "must point to a file",
         () -> Validate.checkPathExistsAsFile(tempDir, "tempDir"));
+
   }
 }

+ 3 - 3
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),

+ 1 - 1
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;

+ 13 - 12
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3BlockManager.java → 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();
   }
 }

+ 16 - 13
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingBlockManager.java → 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);
   }
 

+ 53 - 46
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingInputStream.java → 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());
   }
 }

+ 18 - 14
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InMemoryInputStream.java → 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();
   }
 }

+ 53 - 36
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3PrefetchingInputStream.java → 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;
   }
 

+ 99 - 83
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InputStream.java → 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);
     }
   }
 

+ 36 - 26
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3File.java → 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;
   }

+ 44 - 38
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3Reader.java → 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();
-
-    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;
-        }));
+    Invoker invoker = this.remoteObject.getReadInvoker();
+
+    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);
     }
   }
 }

+ 2 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/package-info.java → 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;

+ 1 - 1
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;
 
 /**

+ 0 - 76
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/EmptyPrefetchingStatistics.java

@@ -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) {
-
-  }
-
-}
-

+ 10 - 10
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3PrefetchingInputStream.java → 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");

+ 1 - 1
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 {

+ 14 - 12
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/MockS3File.java → 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);

+ 77 - 54
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/Fakes.java → 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 ObjectMetadata getObjectMetadata() {
-          ObjectMetadata metadata = new ObjectMetadata();
-          metadata.setHeader("ETag", E_TAG);
-          return metadata;
-        }
-      };
+      @Override
+      public S3ObjectInputStream getObjectContent() {
+        return createS3ObjectInputStream(new byte[8]);
+      }
+
+      @Override
+      public ObjectMetadata getObjectMetadata() {
+        ObjectMetadata metadata = new ObjectMetadata();
+        metadata.setHeader("ETag", E_TAG);
+        return metadata;
+      }
+    };
 
     return new S3AInputStream.InputStreamCallbacks() {
       @Override
@@ -208,8 +214,8 @@ public final class Fakes {
   }
 
 
-  public static S3InputStream createInputStream(
-      Class<? extends S3InputStream> clazz,
+  public static S3ARemoteInputStream createInputStream(
+      Class<? extends S3ARemoteInputStream> clazz,
       ExecutorServiceFuturePool futurePool,
       String bucket,
       String key,
@@ -220,7 +226,8 @@ public final class Fakes {
     org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
 
     S3AFileStatus fileStatus = createFileStatus(key, fileSize);
-    S3ObjectAttributes s3ObjectAttributes = createObjectAttributes(bucket, key, fileSize);
+    S3ObjectAttributes s3ObjectAttributes =
+        createObjectAttributes(bucket, key, fileSize);
     S3AReadOpContext s3AReadOpContext = createReadContext(
         futurePool,
         key,
@@ -228,30 +235,34 @@ public final class Fakes {
         prefetchBlockSize,
         prefetchBlockCount);
 
-    S3AInputStream.InputStreamCallbacks callbacks = createInputStreamCallbacks(bucket, key);
+    S3AInputStream.InputStreamCallbacks callbacks =
+        createInputStreamCallbacks(bucket, key);
     S3AInputStreamStatistics stats =
         s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics();
 
-    if (clazz == TestS3InMemoryInputStream.class) {
-      return new TestS3InMemoryInputStream(s3AReadOpContext, s3ObjectAttributes, callbacks, stats);
-    } else if (clazz == TestS3CachingInputStream.class) {
-      return new TestS3CachingInputStream(s3AReadOpContext, s3ObjectAttributes, callbacks, stats);
+    if (clazz == FakeS3AInMemoryInputStream.class) {
+      return new FakeS3AInMemoryInputStream(s3AReadOpContext,
+          s3ObjectAttributes, callbacks, stats);
+    } else if (clazz == FakeS3ACachingInputStream.class) {
+      return new FakeS3ACachingInputStream(s3AReadOpContext, s3ObjectAttributes,
+          callbacks, stats);
     }
 
     throw new RuntimeException("Unsupported class: " + clazz);
   }
 
-  public static TestS3InMemoryInputStream createS3InMemoryInputStream(
+  public static FakeS3AInMemoryInputStream createS3InMemoryInputStream(
       ExecutorServiceFuturePool futurePool,
       String bucket,
       String key,
       int fileSize) {
 
-    return (TestS3InMemoryInputStream) createInputStream(
-        TestS3InMemoryInputStream.class, futurePool, bucket, key, fileSize, 1, 1);
+    return (FakeS3AInMemoryInputStream) createInputStream(
+        FakeS3AInMemoryInputStream.class, futurePool, bucket, key, fileSize, 1,
+        1);
   }
 
-  public static TestS3CachingInputStream createS3CachingInputStream(
+  public static FakeS3ACachingInputStream createS3CachingInputStream(
       ExecutorServiceFuturePool futurePool,
       String bucket,
       String key,
@@ -259,8 +270,8 @@ public final class Fakes {
       int prefetchBlockSize,
       int prefetchBlockCount) {
 
-    return (TestS3CachingInputStream) createInputStream(
-        TestS3CachingInputStream.class,
+    return (FakeS3ACachingInputStream) createInputStream(
+        FakeS3ACachingInputStream.class,
         futurePool,
         bucket,
         key,
@@ -269,8 +280,10 @@ public final class Fakes {
         prefetchBlockCount);
   }
 
-  public static class TestS3InMemoryInputStream extends S3InMemoryInputStream {
-    public TestS3InMemoryInputStream(
+  public static class FakeS3AInMemoryInputStream
+      extends S3AInMemoryInputStream {
+
+    public FakeS3AInMemoryInputStream(
         S3AReadOpContext context,
         S3ObjectAttributes s3Attributes,
         S3AInputStream.InputStreamCallbacks client,
@@ -279,18 +292,22 @@ public final class Fakes {
     }
 
     @Override
-    protected S3File getS3File() {
+    protected S3ARemoteObject getS3File() {
       randomDelay(200);
-      return new MockS3File((int) this.getS3ObjectAttributes().getLen(), false);
+      return new MockS3ARemoteObject(
+          (int) this.getS3ObjectAttributes().getLen(), false);
     }
   }
 
-  public static class TestS3FilePerBlockCache extends SingleFilePerBlockCache {
+  public static class FakeS3FilePerBlockCache extends SingleFilePerBlockCache {
+
     private final Map<Path, byte[]> files;
+
     private final int readDelay;
+
     private final int writeDelay;
 
-    public TestS3FilePerBlockCache(int readDelay, int writeDelay) {
+    public FakeS3FilePerBlockCache(int readDelay, int writeDelay) {
       super(new EmptyS3AStatisticsContext().newInputStreamStatistics());
       this.files = new ConcurrentHashMap<>();
       this.readDelay = readDelay;
@@ -339,10 +356,12 @@ public final class Fakes {
     }
   }
 
-  public static class TestS3CachingBlockManager extends S3CachingBlockManager {
-    public TestS3CachingBlockManager(
+  public static class FakeS3ACachingBlockManager
+      extends S3ACachingBlockManager {
+
+    public FakeS3ACachingBlockManager(
         ExecutorServiceFuturePool futurePool,
-        S3Reader reader,
+        S3ARemoteObjectReader reader,
         BlockData blockData,
         int bufferPoolSize) {
       super(futurePool, reader, blockData, bufferPoolSize,
@@ -350,7 +369,8 @@ public final class Fakes {
     }
 
     @Override
-    public int read(ByteBuffer buffer, long offset, int size) throws IOException {
+    public int read(ByteBuffer buffer, long offset, int size)
+        throws IOException {
       randomDelay(100);
       return this.getReader().read(buffer, offset, size);
     }
@@ -359,12 +379,13 @@ public final class Fakes {
     protected BlockCache createCache() {
       final int readDelayMs = 50;
       final int writeDelayMs = 200;
-      return new TestS3FilePerBlockCache(readDelayMs, writeDelayMs);
+      return new FakeS3FilePerBlockCache(readDelayMs, writeDelayMs);
     }
   }
 
-  public static class TestS3CachingInputStream extends S3CachingInputStream {
-    public TestS3CachingInputStream(
+  public static class FakeS3ACachingInputStream extends S3ACachingInputStream {
+
+    public FakeS3ACachingInputStream(
         S3AReadOpContext context,
         S3ObjectAttributes s3Attributes,
         S3AInputStream.InputStreamCallbacks client,
@@ -373,18 +394,20 @@ public final class Fakes {
     }
 
     @Override
-    protected S3File getS3File() {
+    protected S3ARemoteObject getS3File() {
       randomDelay(200);
-      return new MockS3File((int) this.getS3ObjectAttributes().getLen(), false);
+      return new MockS3ARemoteObject(
+          (int) this.getS3ObjectAttributes().getLen(), false);
     }
 
     @Override
-    protected S3CachingBlockManager createBlockManager(
+    protected S3ACachingBlockManager createBlockManager(
         ExecutorServiceFuturePool futurePool,
-        S3Reader reader,
+        S3ARemoteObjectReader reader,
         BlockData blockData,
         int bufferPoolSize) {
-      return new TestS3CachingBlockManager(futurePool, reader, blockData, bufferPoolSize);
+      return new FakeS3ACachingBlockManager(futurePool, reader, blockData,
+          bufferPoolSize);
     }
   }
 }

+ 20 - 19
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3BlockManager.java → 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);

+ 81 - 57
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3CachingBlockManager.java → 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());
   }
 }

+ 62 - 34
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3InputStream.java → 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());

+ 26 - 15
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3File.java → 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));
   }
 }

+ 21 - 15
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3Reader.java → 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(
+
+    intercept(
         IllegalArgumentException.class,
-        "'s3File' must not be null",
-        () -> new S3Reader(null));
+        "'remoteObject' must not be null",
+        () -> new S3ARemoteObjectReader(null));
 
-    ExceptionAsserts.assertThrows(
+    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]);
         }
       }

+ 4 - 4
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);
     }
   }