Jelajahi Sumber

HADOOP-18028. High performance S3A input stream (#4752)

This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
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 tahun lalu
induk
melakukan
d81676e126
61 mengubah file dengan 9534 tambahan dan 28 penghapusan
  1. 70 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockCache.java
  2. 250 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockData.java
  3. 145 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManager.java
  4. 425 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockOperations.java
  5. 195 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BoundedResourcePool.java
  6. 319 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferData.java
  7. 323 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferPool.java
  8. 638 0
      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. 70 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ExecutorServiceFuturePool.java
  11. 301 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/FilePosition.java
  12. 67 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/PrefetchingStatistics.java
  13. 71 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ResourcePool.java
  14. 93 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Retryer.java
  15. 354 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java
  16. 399 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Validate.java
  17. 28 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/package-info.java
  18. 40 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java
  19. 63 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/ExceptionAsserts.java
  20. 71 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/SampleDataForTests.java
  21. 98 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockCache.java
  22. 159 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockData.java
  23. 106 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockOperations.java
  24. 147 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBoundedResourcePool.java
  25. 244 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java
  26. 158 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java
  27. 98 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestExecutorServiceFuturePool.java
  28. 195 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestFilePosition.java
  29. 80 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestRetryer.java
  30. 341 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestValidate.java
  31. 25 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
  32. 59 14
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  33. 59 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
  34. 52 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
  35. 12 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
  36. 78 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ABlockManager.java
  37. 96 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingBlockManager.java
  38. 214 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java
  39. 102 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java
  40. 260 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java
  41. 475 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java
  42. 328 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java
  43. 175 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java
  44. 29 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/package-info.java
  45. 2 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java
  46. 36 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java
  47. 24 0
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
  48. 192 0
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md
  49. 189 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java
  50. 12 5
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java
  51. 10 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java
  52. 12 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
  53. 2 2
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java
  54. 114 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java
  55. 413 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java
  56. 87 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java
  57. 361 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java
  58. 279 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java
  59. 88 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java
  60. 112 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java
  61. 9 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java

+ 70 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockCache.java

@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.impl.prefetch;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Provides functionality necessary for caching blocks of data read from FileSystem.
+ */
+public interface BlockCache extends Closeable {
+
+  /**
+   * Indicates whether the given block is in this cache.
+   *
+   * @param blockNumber the id of the given block.
+   * @return true if the given block is in this cache, false otherwise.
+   */
+  boolean containsBlock(int blockNumber);
+
+  /**
+   * Gets the blocks in this cache.
+   *
+   * @return the blocks in this cache.
+   */
+  Iterable<Integer> blocks();
+
+  /**
+   * Gets the number of blocks in this cache.
+   *
+   * @return the number of blocks in this cache.
+   */
+  int size();
+
+  /**
+   * Gets the block having the given {@code blockNumber}.
+   *
+   * @param blockNumber the id of the desired block.
+   * @param buffer contents of the desired block are copied to this buffer.
+   * @throws IOException if there is an error reading the given block.
+   */
+  void get(int blockNumber, ByteBuffer buffer) throws IOException;
+
+  /**
+   * Puts the given block in this cache.
+   *
+   * @param blockNumber the id of the given block.
+   * @param buffer contents of the given block to be added to this cache.
+   * @throws IOException if there is an error writing the given block.
+   */
+  void put(int blockNumber, ByteBuffer buffer) throws IOException;
+}

+ 250 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockData.java

@@ -0,0 +1,250 @@
+/*
+ * 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 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 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). */
+    NOT_READY,
+
+    /** A read of this block has been enqueued in the prefetch queue. */
+    QUEUED,
+
+    /** A read of this block has been enqueued in the prefetch queue. */
+    READY,
+
+    /** This block has been cached in the local disk cache. */
+    CACHED
+  }
+
+  /**
+   * State of all blocks in a file.
+   */
+  private State[] state;
+
+  /**
+   * The size of a file.
+   */
+  private final long fileSize;
+
+  /**
+   * The file is divided into blocks of this size.
+   */
+  private final int blockSize;
+
+  /**
+   * 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) {
+    checkNotNegative(fileSize, "fileSize");
+    if (fileSize == 0) {
+      checkNotNegative(blockSize, "blockSize");
+    } else {
+      checkPositiveInteger(blockSize, "blockSize");
+    }
+
+    this.fileSize = fileSize;
+    this.blockSize = blockSize;
+    this.numBlocks =
+        (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++) {
+      setState(b, State.NOT_READY);
+    }
+  }
+
+  /**
+   * Gets the size of each block.
+   * @return the size of each block.
+   */
+  public int getBlockSize() {
+    return blockSize;
+  }
+
+  /**
+   * Gets the size of the associated file.
+   * @return the size of the associated file.
+   */
+  public long getFileSize() {
+    return fileSize;
+  }
+
+  /**
+   * Gets the number of blocks in the associated file.
+   * @return the number of blocks in the associated file.
+   */
+  public int getNumBlocks() {
+    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 (fileSize == 0) {
+      return false;
+    }
+
+    throwIfInvalidBlockNumber(blockNumber);
+
+    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 / 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 (fileSize == 0) {
+      return 0;
+    }
+
+    if (isLastBlock(blockNumber)) {
+      return (int) (fileSize - (((long) blockSize) * (numBlocks - 1)));
+    } else {
+      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 < 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) 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 - 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 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);
+
+    state[blockNumber] = blockState;
+  }
+
+  // Debug helper.
+  public String getStateString() {
+    StringBuilder sb = new StringBuilder();
+    int blockNumber = 0;
+    while (blockNumber < numBlocks) {
+      State tstate = getState(blockNumber);
+      int endBlockNumber = blockNumber;
+      while ((endBlockNumber < numBlocks) && (getState(endBlockNumber)
+          == tstate)) {
+        endBlockNumber++;
+      }
+      sb.append(
+          String.format("[%03d ~ %03d] %s%n", blockNumber, endBlockNumber - 1,
+              tstate));
+      blockNumber = endBlockNumber;
+    }
+    return sb.toString();
+  }
+
+  private void throwIfInvalidBlockNumber(int blockNumber) {
+    checkWithinRange(blockNumber, "blockNumber", 0, numBlocks - 1);
+  }
+
+  private void throwIfInvalidOffset(long offset) {
+    checkWithinRange(offset, "offset", 0, fileSize - 1);
+  }
+}

+ 145 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManager.java

@@ -0,0 +1,145 @@
+/*
+ * 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.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.
+ *
+ * This class is the simplest form of a {@code BlockManager} that does
+ * perform prefetching or caching.
+ */
+public abstract class BlockManager implements Closeable {
+
+  /**
+   * Information about each block of the underlying file.
+   */
+  private final BlockData blockData;
+
+  /**
+   * Constructs an instance of {@code BlockManager}.
+   *
+   * @param blockData information about each block of the underlying file.
+   *
+   * @throws IllegalArgumentException if blockData is null.
+   */
+  public BlockManager(BlockData blockData) {
+    checkNotNull(blockData, "blockData");
+
+    this.blockData = blockData;
+  }
+
+  /**
+   * Gets block data information.
+   *
+   * @return instance of {@code BlockData}.
+   */
+  public BlockData getBlockData() {
+    return blockData;
+  }
+
+  /**
+   * Gets the block having the given {@code blockNumber}.
+   *
+   * The entire block is read into memory and returned as a {@code BufferData}.
+   * The blocks are treated as a limited resource and must be released when
+   * one is done reading them.
+   *
+   * @param blockNumber the number of the block to be read and returned.
+   * @return {@code BufferData} having data from the given block.
+   *
+   * @throws IOException if there an error reading the given block.
+   * @throws IllegalArgumentException if blockNumber is negative.
+   */
+  public BufferData get(int blockNumber) throws IOException {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    int size = blockData.getSize(blockNumber);
+    ByteBuffer buffer = ByteBuffer.allocate(size);
+    long startOffset = blockData.getStartOffset(blockNumber);
+    read(buffer, startOffset, size);
+    buffer.flip();
+    return new BufferData(blockNumber, buffer);
+  }
+
+  /**
+   * Reads into the given {@code buffer} {@code size} bytes from the underlying file
+   * starting at {@code startOffset}.
+   *
+   * @param buffer the buffer to read data in to.
+   * @param startOffset the offset at which reading starts.
+   * @param size the number bytes to read.
+   * @return number of bytes read.
+   * @throws IOException if there an error reading the given block.
+   */
+  public abstract int read(ByteBuffer buffer, long startOffset, int size) throws IOException;
+
+  /**
+   * Releases resources allocated to the given block.
+   *
+   * @param data the {@code BufferData} to release.
+   *
+   * @throws IllegalArgumentException if data is null.
+   */
+  public void release(BufferData data) {
+    checkNotNull(data, "data");
+
+    // Do nothing because we allocate a new buffer each time.
+  }
+
+  /**
+   * Requests optional prefetching of the given block.
+   *
+   * @param blockNumber the id of the block to prefetch.
+   *
+   * @throws IllegalArgumentException if blockNumber is negative.
+   */
+  public void requestPrefetch(int blockNumber) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    // Do nothing because we do not support prefetches.
+  }
+
+  /**
+   * Requests cancellation of any previously issued prefetch requests.
+   */
+  public void cancelPrefetches() {
+    // Do nothing because we do not support prefetches.
+  }
+
+  /**
+   * Requests that the given block should be copied to the cache. Optional operation.
+   *
+   * @param data the {@code BufferData} instance to optionally cache.
+   */
+  public void requestCaching(BufferData data) {
+    // Do nothing because we do not support caching.
+  }
+
+  @Override
+  public void close() {
+  }
+}

+ 425 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockOperations.java

@@ -0,0 +1,425 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.DoubleSummaryStatistics;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+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}.
+ * It is separated out in its own file due to its size.
+ *
+ * This class is used for debugging/logging. Calls to this class
+ * can be safely removed without affecting the overall operation.
+ */
+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),
+    CLOSE("CX", "close", false),
+    CACHE_PUT("C+", "putC", true),
+    GET_CACHED("GC", "getCached", true),
+    GET_PREFETCHED("GP", "getPrefetched", true),
+    GET_READ("GR", "getRead", true),
+    PREFETCH("PF", "prefetch", true),
+    RELEASE("RL", "release", true),
+    REQUEST_CACHING("RC", "requestCaching", true),
+    REQUEST_PREFETCH("RP", "requestPrefetch", true);
+
+    private String shortName;
+    private String name;
+    private boolean hasBlock;
+
+    Kind(String shortName, String name, boolean hasBlock) {
+      this.shortName = shortName;
+      this.name = name;
+      this.hasBlock = hasBlock;
+    }
+
+    private static Map<String, Kind> shortNameToKind = new HashMap<>();
+
+    public static Kind fromShortName(String shortName) {
+      if (shortNameToKind.isEmpty()) {
+        for (Kind kind : Kind.values()) {
+          shortNameToKind.put(kind.shortName, kind);
+        }
+      }
+      return shortNameToKind.get(shortName);
+    }
+  }
+
+  public static class Operation {
+    private final Kind kind;
+    private final int blockNumber;
+    private final long timestamp;
+
+    public Operation(Kind kind, int blockNumber) {
+      this.kind = kind;
+      this.blockNumber = blockNumber;
+      this.timestamp = System.nanoTime();
+    }
+
+    public Kind getKind() {
+      return kind;
+    }
+
+    public int getBlockNumber() {
+      return blockNumber;
+    }
+
+    public long getTimestamp() {
+      return timestamp;
+    }
+
+    public void getSummary(StringBuilder sb) {
+      if (kind.hasBlock) {
+        sb.append(String.format("%s(%d)", kind.shortName, blockNumber));
+      } else {
+        sb.append(String.format("%s", kind.shortName));
+      }
+    }
+
+    public String getDebugInfo() {
+      if (kind.hasBlock) {
+        return String.format("--- %s(%d)", kind.name, blockNumber);
+      } else {
+        return String.format("... %s()", kind.name);
+      }
+    }
+  }
+
+  public static class End extends Operation {
+    private Operation op;
+
+    public End(Operation op) {
+      super(op.kind, op.blockNumber);
+      this.op = op;
+    }
+
+    @Override
+    public void getSummary(StringBuilder sb) {
+      sb.append("E");
+      super.getSummary(sb);
+    }
+
+    @Override
+    public String getDebugInfo() {
+      return "***" + super.getDebugInfo().substring(3);
+    }
+
+    public double duration() {
+      return (getTimestamp() - op.getTimestamp()) / 1e9;
+    }
+  }
+
+  private ArrayList<Operation> ops;
+  private boolean debugMode;
+
+  public BlockOperations() {
+    this.ops = new ArrayList<>();
+  }
+
+  public synchronized void setDebug(boolean state) {
+    debugMode = state;
+  }
+
+  private synchronized Operation add(Operation op) {
+    if (debugMode) {
+      LOG.info(op.getDebugInfo());
+    }
+    ops.add(op);
+    return op;
+  }
+
+  public Operation getPrefetched(int blockNumber) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    return add(new Operation(Kind.GET_PREFETCHED, blockNumber));
+  }
+
+  public Operation getCached(int blockNumber) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    return add(new Operation(Kind.GET_CACHED, blockNumber));
+  }
+
+  public Operation getRead(int blockNumber) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    return add(new Operation(Kind.GET_READ, blockNumber));
+  }
+
+  public Operation release(int blockNumber) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    return add(new Operation(Kind.RELEASE, blockNumber));
+  }
+
+  public Operation requestPrefetch(int blockNumber) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    return add(new Operation(Kind.REQUEST_PREFETCH, blockNumber));
+  }
+
+  public Operation prefetch(int blockNumber) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    return add(new Operation(Kind.PREFETCH, blockNumber));
+  }
+
+  public Operation cancelPrefetches() {
+    return add(new Operation(Kind.CANCEL_PREFETCHES, -1));
+  }
+
+  public Operation close() {
+    return add(new Operation(Kind.CLOSE, -1));
+  }
+
+  public Operation requestCaching(int blockNumber) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    return add(new Operation(Kind.REQUEST_CACHING, blockNumber));
+  }
+
+  public Operation addToCache(int blockNumber) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    return add(new Operation(Kind.CACHE_PUT, blockNumber));
+  }
+
+  public Operation end(Operation op) {
+    return add(new End(op));
+  }
+
+  private static void append(StringBuilder sb, String format, Object... args) {
+    sb.append(String.format(format, args));
+  }
+
+  public synchronized String getSummary(boolean showDebugInfo) {
+    StringBuilder sb = new StringBuilder();
+    for (Operation op : ops) {
+      if (op != null) {
+        if (showDebugInfo) {
+          sb.append(op.getDebugInfo());
+          sb.append("\n");
+        } else {
+          op.getSummary(sb);
+          sb.append(";");
+        }
+      }
+    }
+
+    sb.append("\n");
+    getDurationInfo(sb);
+
+    return sb.toString();
+  }
+
+  public synchronized void getDurationInfo(StringBuilder sb) {
+    Map<Kind, DoubleSummaryStatistics> durations = new HashMap<>();
+    for (Operation op : ops) {
+      if (op instanceof End) {
+        End endOp = (End) op;
+        DoubleSummaryStatistics stats = durations.get(endOp.getKind());
+        if (stats == null) {
+          stats = new DoubleSummaryStatistics();
+          durations.put(endOp.getKind(), stats);
+        }
+        stats.accept(endOp.duration());
+      }
+    }
+
+    List<Kind> kinds = Arrays.asList(
+        Kind.GET_CACHED,
+        Kind.GET_PREFETCHED,
+        Kind.GET_READ,
+        Kind.CACHE_PUT,
+        Kind.PREFETCH,
+        Kind.REQUEST_CACHING,
+        Kind.REQUEST_PREFETCH,
+        Kind.CANCEL_PREFETCHES,
+        Kind.RELEASE,
+        Kind.CLOSE
+    );
+
+    for (Kind kind : kinds) {
+      append(sb, "%-18s : ", kind);
+      DoubleSummaryStatistics stats = durations.get(kind);
+      if (stats == null) {
+        append(sb, "--\n");
+      } else {
+        append(
+            sb,
+            "#ops = %3d, total = %5.1f, min: %3.1f, avg: %3.1f, max: %3.1f\n",
+            stats.getCount(),
+            stats.getSum(),
+            stats.getMin(),
+            stats.getAverage(),
+            stats.getMax());
+      }
+    }
+  }
+
+  public synchronized void analyze(StringBuilder sb) {
+    Map<Integer, List<Operation>> blockOps = new HashMap<>();
+
+    // Group-by block number.
+    for (Operation op : ops) {
+      if (op.blockNumber < 0) {
+        continue;
+      }
+
+      List<Operation> perBlockOps;
+      if (!blockOps.containsKey(op.blockNumber)) {
+        perBlockOps = new ArrayList<>();
+        blockOps.put(op.blockNumber, perBlockOps);
+      }
+
+      perBlockOps = blockOps.get(op.blockNumber);
+      perBlockOps.add(op);
+    }
+
+    List<Integer> prefetchedNotUsed = new ArrayList<>();
+    List<Integer> cachedNotUsed = new ArrayList<>();
+
+    for (Map.Entry<Integer, List<Operation>> entry : blockOps.entrySet()) {
+      Integer blockNumber = entry.getKey();
+      List<Operation> perBlockOps = entry.getValue();
+      Map<Kind, Integer> kindCounts = new HashMap<>();
+      Map<Kind, Integer> endKindCounts = new HashMap<>();
+
+      for (Operation op : perBlockOps) {
+        if (op instanceof End) {
+          int endCount = endKindCounts.getOrDefault(op.kind, 0) + 1;
+          endKindCounts.put(op.kind, endCount);
+        } else {
+          int count = kindCounts.getOrDefault(op.kind, 0) + 1;
+          kindCounts.put(op.kind, count);
+        }
+      }
+
+      for (Kind kind : kindCounts.keySet()) {
+        int count = kindCounts.getOrDefault(kind, 0);
+        int endCount = endKindCounts.getOrDefault(kind, 0);
+        if (count != endCount) {
+          append(sb, "[%d] %s : #ops(%d) != #end-ops(%d)\n", blockNumber, kind, count, endCount);
+        }
+
+        if (count > 1) {
+          append(sb, "[%d] %s = %d\n", blockNumber, kind, count);
+        }
+      }
+
+      int prefetchCount = kindCounts.getOrDefault(Kind.PREFETCH, 0);
+      int getPrefetchedCount = kindCounts.getOrDefault(Kind.GET_PREFETCHED, 0);
+      if ((prefetchCount > 0) && (getPrefetchedCount < prefetchCount)) {
+        prefetchedNotUsed.add(blockNumber);
+      }
+
+      int cacheCount = kindCounts.getOrDefault(Kind.CACHE_PUT, 0);
+      int getCachedCount = kindCounts.getOrDefault(Kind.GET_CACHED, 0);
+      if ((cacheCount > 0) && (getCachedCount < cacheCount)) {
+        cachedNotUsed.add(blockNumber);
+      }
+    }
+
+    if (!prefetchedNotUsed.isEmpty()) {
+      append(sb, "Prefetched but not used: %s\n", getIntList(prefetchedNotUsed));
+    }
+
+    if (!cachedNotUsed.isEmpty()) {
+      append(sb, "Cached but not used: %s\n", getIntList(cachedNotUsed));
+    }
+  }
+
+  private static String getIntList(Iterable<Integer> nums) {
+    List<String> numList = new ArrayList<>();
+    for (Integer n : nums) {
+      numList.add(n.toString());
+    }
+    return String.join(", ", numList);
+  }
+
+  public static BlockOperations fromSummary(String summary) {
+    BlockOperations ops = new BlockOperations();
+    ops.setDebug(true);
+    Pattern blockOpPattern = Pattern.compile("([A-Z+]+)(\\(([0-9]+)?\\))?");
+    String[] tokens = summary.split(";");
+    for (String token : tokens) {
+      Matcher matcher = blockOpPattern.matcher(token);
+      if (!matcher.matches()) {
+        String message = String.format("Unknown summary format: %s", token);
+        throw new IllegalArgumentException(message);
+      }
+
+      String shortName = matcher.group(1);
+      String blockNumberStr = matcher.group(3);
+      int blockNumber = (blockNumberStr == null) ? -1 : Integer.parseInt(blockNumberStr);
+      Kind kind = Kind.fromShortName(shortName);
+      Kind endKind = null;
+      if (kind == null) {
+        if (shortName.charAt(0) == 'E') {
+          endKind = Kind.fromShortName(shortName.substring(1));
+        }
+      }
+
+      if (kind == null && endKind == null) {
+        String message = String.format("Unknown short name: %s (token = %s)", shortName, token);
+        throw new IllegalArgumentException(message);
+      }
+
+      if (kind != null) {
+        ops.add(new Operation(kind, blockNumber));
+      } else {
+        Operation op = null;
+        for (int i = ops.ops.size() - 1; i >= 0; i--) {
+          op = ops.ops.get(i);
+          if ((op.blockNumber == blockNumber) && (op.kind == endKind) && !(op instanceof End)) {
+            ops.add(new End(op));
+            break;
+          }
+        }
+
+        if (op == null) {
+          LOG.warn("Start op not found: {}({})", endKind, blockNumber);
+        }
+      }
+    }
+
+    return ops;
+  }
+}

+ 195 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BoundedResourcePool.java

@@ -0,0 +1,195 @@
+/*
+ * 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.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.
+   */
+  private final int size;
+
+  /**
+   * Items currently available in the pool.
+   */
+  private ArrayBlockingQueue<T> items;
+
+  /**
+   * Items that have been created so far (regardless of whether they are currently available).
+   */
+  private Set<T> createdItems;
+
+  /**
+   * Constructs a resource pool of the given size.
+   *
+   * @param size the size of this pool. Cannot be changed post creation.
+   *
+   * @throws IllegalArgumentException if size is zero or negative.
+   */
+  public BoundedResourcePool(int size) {
+    Validate.checkPositiveInteger(size, "size");
+
+    this.size = size;
+    this.items = new ArrayBlockingQueue<>(size);
+
+    // The created items are identified based on their object reference.
+    this.createdItems = Collections.newSetFromMap(new IdentityHashMap<T, Boolean>());
+  }
+
+  /**
+   * Acquires a resource blocking if necessary until one becomes available.
+   */
+  @Override
+  public T acquire() {
+    return this.acquireHelper(true);
+  }
+
+  /**
+   * Acquires a resource blocking if one is immediately available. Otherwise returns null.
+   */
+  @Override
+  public T tryAcquire() {
+    return this.acquireHelper(false);
+  }
+
+  /**
+   * Releases a previously acquired resource.
+   *
+   * @throws IllegalArgumentException if item is null.
+   */
+  @Override
+  public void release(T item) {
+    checkNotNull(item, "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 items.contains() because that check is not based on reference equality.
+    for (T entry : items) {
+      if (entry == item) {
+        return;
+      }
+    }
+
+    try {
+      items.put(item);
+    } catch (InterruptedException e) {
+      throw new IllegalStateException("release() should never block", e);
+    }
+  }
+
+  @Override
+  public synchronized void close() {
+    for (T item : createdItems) {
+      close(item);
+    }
+
+    items.clear();
+    items = null;
+
+    createdItems.clear();
+    createdItems = null;
+  }
+
+  /**
+   * Derived classes may implement a way to cleanup each item.
+   */
+  @Override
+  protected synchronized void close(T item) {
+    // Do nothing in this class. Allow overriding classes to take any cleanup action.
+  }
+
+  /**
+   * Number of items created so far. Mostly for testing purposes.
+   * @return the count.
+   */
+  public int numCreated() {
+    synchronized (createdItems) {
+      return createdItems.size();
+    }
+  }
+
+  /**
+   * Number of items available to be acquired. Mostly for testing purposes.
+   * @return the number available.
+   */
+  public synchronized int numAvailable() {
+    return (size - numCreated()) + items.size();
+  }
+
+  // For debugging purposes.
+  @Override
+  public synchronized String toString() {
+    return String.format(
+        "size = %d, #created = %d, #in-queue = %d, #available = %d",
+        size, numCreated(), items.size(), numAvailable());
+  }
+
+  /**
+   * Derived classes must implement a way to create an instance of a resource.
+   */
+  protected abstract T createNew();
+
+  private T acquireHelper(boolean canBlock) {
+
+    // Prefer reusing an item if one is available.
+    // That avoids unnecessarily creating new instances.
+    T result = items.poll();
+    if (result != null) {
+      return result;
+    }
+
+    synchronized (createdItems) {
+      // Create a new instance if allowed by the capacity of this pool.
+      if (createdItems.size() < size) {
+        T item = createNew();
+        createdItems.add(item);
+        return item;
+      }
+    }
+
+    if (canBlock) {
+      try {
+        // Block for an instance to be available.
+        return items.take();
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        return null;
+      }
+    } else {
+      return null;
+    }
+  }
+}

+ 319 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferData.java

@@ -0,0 +1,319 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Future;
+import java.util.zip.CRC32;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Holds the state of a ByteBuffer that is in use by {@code CachingBlockManager}.
+ *
+ * This class is not meant to be of general use. It exists into its own file due to its size.
+ * We use the term block and buffer interchangeably in this file because one buffer
+ * holds exactly one block of data.
+ *
+ * Holding all of the state associated with a block allows us to validate and control
+ * state transitions in a synchronized fashion.
+ */
+public final class BufferData {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BufferData.class);
+
+  public enum State {
+    /**
+     * Unknown / invalid state.
+     */
+    UNKNOWN,
+
+    /**
+     * Buffer has been acquired but has no data.
+     */
+    BLANK,
+
+    /**
+     * This block is being prefetched.
+     */
+    PREFETCHING,
+
+    /**
+     * This block is being added to the local cache.
+     */
+    CACHING,
+
+    /**
+     * 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.
+     */
+    DONE
+  }
+
+  /**
+   * Number of the block associated with this buffer.
+   */
+  private final int blockNumber;
+
+  /**
+   * The buffer associated with this block.
+   */
+  private ByteBuffer buffer;
+
+  /**
+   * Current state of this block.
+   */
+  private volatile State state;
+
+  /**
+   * 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.
+   */
+  private long checksum = 0;
+
+  /**
+   * Constructs an instances of this class.
+   *
+   * @param blockNumber Number of the block associated with this buffer.
+   * @param buffer The buffer associated with this block.
+   *
+   * @throws IllegalArgumentException if blockNumber is negative.
+   * @throws IllegalArgumentException if buffer is null.
+   */
+  public BufferData(int blockNumber, ByteBuffer buffer) {
+    Validate.checkNotNegative(blockNumber, "blockNumber");
+    Validate.checkNotNull(buffer, "buffer");
+
+    this.blockNumber = blockNumber;
+    this.buffer = buffer;
+    this.state = State.BLANK;
+  }
+
+  /**
+   * Gets the id of this block.
+   *
+   * @return the id of this block.
+   */
+  public int getBlockNumber() {
+    return this.blockNumber;
+  }
+
+  /**
+   * Gets the buffer associated with this block.
+   *
+   * @return the buffer associated with this block.
+   */
+  public ByteBuffer getBuffer() {
+    return this.buffer;
+  }
+
+  /**
+   * Gets the state of this block.
+   *
+   * @return the state of this block.
+   */
+  public State getState() {
+    return this.state;
+  }
+
+  /**
+   * Gets the checksum of data in this block.
+   *
+   * @return the checksum of data in this block.
+   */
+  public long getChecksum() {
+    return this.checksum;
+  }
+
+  /**
+   * Computes CRC32 checksum of the given buffer's contents.
+   *
+   * @param buffer the buffer whose content's checksum is to be computed.
+   * @return the computed checksum.
+   */
+  public static long getChecksum(ByteBuffer buffer) {
+    ByteBuffer tempBuffer = buffer.duplicate();
+    tempBuffer.rewind();
+    CRC32 crc32 = new CRC32();
+    crc32.update(tempBuffer);
+    return crc32.getValue();
+  }
+
+  public synchronized Future<Void> getActionFuture() {
+    return this.action;
+  }
+
+  /**
+   * Indicates that a prefetch operation is in progress.
+   *
+   * @param actionFuture the {@code Future} of a prefetch action.
+   *
+   * @throws IllegalArgumentException if actionFuture is null.
+   */
+  public synchronized void setPrefetch(Future<Void> actionFuture) {
+    Validate.checkNotNull(actionFuture, "actionFuture");
+
+    this.updateState(State.PREFETCHING, State.BLANK);
+    this.action = actionFuture;
+  }
+
+  /**
+   * Indicates that a caching operation is in progress.
+   *
+   * @param actionFuture the {@code Future} of a caching action.
+   *
+   * @throws IllegalArgumentException if actionFuture is null.
+   */
+  public synchronized void setCaching(Future<Void> actionFuture) {
+    Validate.checkNotNull(actionFuture, "actionFuture");
+
+    this.throwIfStateIncorrect(State.PREFETCHING, State.READY);
+    this.state = State.CACHING;
+    this.action = actionFuture;
+  }
+
+  /**
+   * Marks the completion of reading data into the buffer.
+   * The buffer cannot be modified once in this state.
+   *
+   * @param expectedCurrentState the collection of states from which transition to READY is allowed.
+   */
+  public synchronized void setReady(State... expectedCurrentState) {
+    if (this.checksum != 0) {
+      throw new IllegalStateException("Checksum cannot be changed once set");
+    }
+
+    this.buffer = this.buffer.asReadOnlyBuffer();
+    this.checksum = getChecksum(this.buffer);
+    this.buffer.rewind();
+    this.updateState(State.READY, expectedCurrentState);
+  }
+
+  /**
+   * Indicates that this block is no longer of use and can be reclaimed.
+   */
+  public synchronized void setDone() {
+    if (this.checksum != 0) {
+      if (getChecksum(this.buffer) != this.checksum) {
+        throw new IllegalStateException("checksum changed after setReady()");
+      }
+    }
+    this.state = State.DONE;
+    this.action = null;
+  }
+
+  /**
+   * 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.
+   *
+   * @throws IllegalArgumentException if newState is null.
+   * @throws IllegalArgumentException if expectedCurrentState is null.
+   */
+  public synchronized void updateState(State newState,
+      State... expectedCurrentState) {
+    Validate.checkNotNull(newState, "newState");
+    Validate.checkNotNull(expectedCurrentState, "expectedCurrentState");
+
+    this.throwIfStateIncorrect(expectedCurrentState);
+    this.state = newState;
+  }
+
+  /**
+   * Helper that asserts the current state is one of the expected values.
+   *
+   * @param states the collection of allowed states.
+   *
+   * @throws IllegalArgumentException if states is null.
+   */
+  public void throwIfStateIncorrect(State... states) {
+    Validate.checkNotNull(states, "states");
+
+    if (this.stateEqualsOneOf(states)) {
+      return;
+    }
+
+    List<String> statesStr = new ArrayList<String>();
+    for (State s : states) {
+      statesStr.add(s.toString());
+    }
+
+    String message = String.format(
+        "Expected buffer state to be '%s' but found: %s",
+        String.join(" or ", statesStr), this);
+    throw new IllegalStateException(message);
+  }
+
+  public boolean stateEqualsOneOf(State... states) {
+    State currentState = this.state;
+
+    for (State s : states) {
+      if (currentState == s) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  public String toString() {
+
+    return String.format(
+        "[%03d] id: %03d, %s: buf: %s, checksum: %d, future: %s",
+        this.blockNumber,
+        System.identityHashCode(this),
+        this.state,
+        this.getBufferStr(this.buffer),
+        this.checksum,
+        this.getFutureStr(this.action));
+  }
+
+  private String getFutureStr(Future<Void> f) {
+    if (f == null) {
+      return "--";
+    } else {
+      return this.action.isDone() ? "done" : "not done";
+    }
+  }
+
+  private String getBufferStr(ByteBuffer buf) {
+    if (buf == null) {
+      return "--";
+    } else {
+      return String.format(
+          "(id = %d, pos = %d, lim = %d)",
+          System.identityHashCode(buf),
+          buf.position(), buf.limit());
+    }
+  }
+}

+ 323 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferPool.java

@@ -0,0 +1,323 @@
+/*
+ * 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.io.Closeable;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+
+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.
+   */
+  private final int size;
+
+  /**
+   * 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
+  */
+
+
+  /**
+   * Underlying bounded resource pool.
+   */
+  private BoundedResourcePool<ByteBuffer> 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) {
+    Validate.checkPositiveInteger(size, "size");
+    Validate.checkPositiveInteger(bufferSize, "bufferSize");
+
+    this.size = size;
+    this.bufferSize = bufferSize;
+    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;
+      }
+    };
+  }
+
+  /**
+   * Gets a list of all blocks in this pool.
+   * @return a list of all blocks in this pool.
+   */
+  public List<BufferData> getAll() {
+    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}.
+   */
+  public synchronized BufferData acquire(int blockNumber) {
+    BufferData data;
+    final int maxRetryDelayMs = 600 * 1000;
+    final int statusUpdateDelayMs = 120 * 1000;
+    Retryer retryer = new Retryer(10, maxRetryDelayMs, statusUpdateDelayMs);
+
+    do {
+      if (retryer.updateStatus()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("waiting to acquire block: {}", blockNumber);
+          LOG.debug("state = {}", this);
+        }
+        releaseReadyBlock(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);
+      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 acquireHelper(blockNumber, false);
+  }
+
+  private synchronized BufferData acquireHelper(int blockNumber,
+      boolean canBlock) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    releaseDoneBlocks();
+
+    BufferData data = find(blockNumber);
+    if (data != null) {
+      return data;
+    }
+
+    ByteBuffer buffer = canBlock ? pool.acquire() : pool.tryAcquire();
+    if (buffer == null) {
+      return null;
+    }
+
+    buffer.clear();
+    data = new BufferData(blockNumber, buffer.duplicate());
+
+    synchronized (allocated) {
+      checkState(find(blockNumber) == null, "buffer data already exists");
+
+      allocated.put(data, buffer);
+    }
+
+    return data;
+  }
+
+  /**
+   * Releases resources for any blocks marked as 'done'.
+   */
+  private synchronized void releaseDoneBlocks() {
+    for (BufferData data : getAll()) {
+      if (data.stateEqualsOneOf(BufferData.State.DONE)) {
+        release(data);
+      }
+    }
+  }
+
+  /**
+   * If no blocks were released after calling releaseDoneBlocks() a few times,
+   * we may end up waiting forever. To avoid that situation, we try releasing
+   * a 'ready' block farthest away from the given block.
+   */
+  private synchronized void releaseReadyBlock(int blockNumber) {
+    BufferData releaseTarget = null;
+    for (BufferData data : getAll()) {
+      if (data.stateEqualsOneOf(BufferData.State.READY)) {
+        if (releaseTarget == null) {
+          releaseTarget = data;
+        } else {
+          if (distance(data, blockNumber) > distance(releaseTarget,
+              blockNumber)) {
+            releaseTarget = data;
+          }
+        }
+      }
+    }
+
+    if (releaseTarget != null) {
+      LOG.warn("releasing 'ready' block: {}", releaseTarget);
+      releaseTarget.setDone();
+    }
+  }
+
+  private int distance(BufferData data, int blockNumber) {
+    return Math.abs(data.getBlockNumber() - blockNumber);
+  }
+
+  /**
+   * 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) {
+    checkNotNull(data, "data");
+
+    synchronized (data) {
+      checkArgument(
+          canRelease(data),
+          String.format("Unable to release buffer: %s", data));
+
+      ByteBuffer buffer = allocated.get(data);
+      if (buffer == null) {
+        // Likely released earlier.
+        return;
+      }
+      buffer.clear();
+      pool.release(buffer);
+      allocated.remove(data);
+    }
+
+    releaseDoneBlocks();
+  }
+
+  @Override
+  public synchronized void close() {
+    for (BufferData data : getAll()) {
+      Future<Void> actionFuture = data.getActionFuture();
+      if (actionFuture != null) {
+        actionFuture.cancel(true);
+      }
+    }
+
+    int currentPoolSize = pool.numCreated();
+
+    pool.close();
+    pool = null;
+
+    allocated.clear();
+    allocated = null;
+
+    prefetchingStatistics.memoryFreed(currentPoolSize * bufferSize);
+  }
+
+  // For debugging purposes.
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(pool.toString());
+    sb.append("\n");
+    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");
+    }
+
+    return sb.toString();
+  }
+
+  // Number of ByteBuffers created so far.
+  public synchronized int numCreated() {
+    return pool.numCreated();
+  }
+
+  // Number of ByteBuffers available to be acquired.
+  public synchronized int numAvailable() {
+    releaseDoneBlocks();
+    return pool.numAvailable();
+  }
+
+  private BufferData find(int blockNumber) {
+    synchronized (allocated) {
+      for (BufferData data : allocated.keySet()) {
+        if ((data.getBlockNumber() == blockNumber)
+            && !data.stateEqualsOneOf(BufferData.State.DONE)) {
+          return data;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private boolean canRelease(BufferData data) {
+    return data.stateEqualsOneOf(
+        BufferData.State.DONE,
+        BufferData.State.READY);
+  }
+}

+ 638 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java

@@ -0,0 +1,638 @@
+/*
+ * 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.io.IOException;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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;
+
+/**
+ * Provides read access to the underlying file one block at a time.
+ * Improve read performance by prefetching and locall caching blocks.
+ */
+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.
+   */
+  private final ExecutorServiceFuturePool futurePool;
+
+  /**
+   * 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.
+   */
+  private final int bufferPoolSize;
+
+  /**
+   * Local block cache.
+   */
+  private BlockCache cache;
+
+  /**
+   * Error counts. For testing purposes.
+   */
+  private final AtomicInteger numCachingErrors;
+  private final AtomicInteger numReadErrors;
+
+  /**
+   * 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.
+   */
+  private static final int SLOW_CACHING_THRESHOLD = 5;
+
+  /**
+   * Once set to true, any further caching requests will be ignored.
+   */
+  private final AtomicBoolean cachingDisabled;
+
+  private final PrefetchingStatistics prefetchingStatistics;
+
+  /**
+   * Constructs an instance of a {@code CachingBlockManager}.
+   *
+   * @param futurePool asynchronous tasks are performed in this pool.
+   * @param blockData information about each block of the underlying file.
+   * @param bufferPoolSize size of the in-memory cache in terms of number of blocks.
+   * @param prefetchingStatistics statistics for this stream.
+   *
+   * @throws IllegalArgumentException if bufferPoolSize is zero or negative.
+   */
+  public CachingBlockManager(
+      ExecutorServiceFuturePool futurePool,
+      BlockData blockData,
+      int bufferPoolSize,
+      PrefetchingStatistics prefetchingStatistics) {
+    super(blockData);
+
+    Validate.checkPositiveInteger(bufferPoolSize, "bufferPoolSize");
+
+    this.futurePool = requireNonNull(futurePool);
+    this.bufferPoolSize = bufferPoolSize;
+    this.numCachingErrors = new AtomicInteger();
+    this.numReadErrors = new AtomicInteger();
+    this.cachingDisabled = new AtomicBoolean();
+    this.prefetchingStatistics = requireNonNull(prefetchingStatistics);
+
+    if (this.getBlockData().getFileSize() > 0) {
+      this.bufferPool = new BufferPool(bufferPoolSize, this.getBlockData().getBlockSize(),
+          this.prefetchingStatistics);
+      this.cache = this.createCache();
+    }
+
+    this.ops = new BlockOperations();
+    this.ops.setDebug(false);
+  }
+
+  /**
+   * Gets the block having the given {@code blockNumber}.
+   *
+   * @throws IllegalArgumentException if blockNumber is negative.
+   */
+  @Override
+  public BufferData get(int blockNumber) throws IOException {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    BufferData data;
+    final int maxRetryDelayMs = bufferPoolSize * 120 * 1000;
+    final int statusUpdateDelayMs = 120 * 1000;
+    Retryer retryer = new Retryer(10, maxRetryDelayMs, statusUpdateDelayMs);
+    boolean done;
+
+    do {
+      if (closed) {
+        throw new IOException("this stream is already closed");
+      }
+
+      data = bufferPool.acquire(blockNumber);
+      done = getInternal(data);
+
+      if (retryer.updateStatus()) {
+        LOG.warn("waiting to get block: {}", blockNumber);
+        LOG.info("state = {}", this.toString());
+      }
+    }
+    while (!done && retryer.continueRetry());
+
+    if (done) {
+      return data;
+    } else {
+      String message = String.format("Wait failed for get(%d)", blockNumber);
+      throw new IllegalStateException(message);
+    }
+  }
+
+  private boolean getInternal(BufferData data) throws IOException {
+    Validate.checkNotNull(data, "data");
+
+    // Opportunistic check without locking.
+    if (data.stateEqualsOneOf(
+        BufferData.State.PREFETCHING,
+        BufferData.State.CACHING,
+        BufferData.State.DONE)) {
+      return false;
+    }
+
+    synchronized (data) {
+      // Reconfirm state after locking.
+      if (data.stateEqualsOneOf(
+          BufferData.State.PREFETCHING,
+          BufferData.State.CACHING,
+          BufferData.State.DONE)) {
+        return false;
+      }
+
+      int blockNumber = data.getBlockNumber();
+      if (data.getState() == BufferData.State.READY) {
+        BlockOperations.Operation op = ops.getPrefetched(blockNumber);
+        ops.end(op);
+        return true;
+      }
+
+      data.throwIfStateIncorrect(BufferData.State.BLANK);
+      read(data);
+      return true;
+    }
+  }
+
+  /**
+   * Releases resources allocated to the given block.
+   *
+   * @throws IllegalArgumentException if data is null.
+   */
+  @Override
+  public void release(BufferData data) {
+    if (closed) {
+      return;
+    }
+
+    Validate.checkNotNull(data, "data");
+
+    BlockOperations.Operation op = ops.release(data.getBlockNumber());
+    bufferPool.release(data);
+    ops.end(op);
+  }
+
+  @Override
+  public synchronized void close() {
+    if (closed) {
+      return;
+    }
+
+    closed = true;
+
+    final BlockOperations.Operation op = ops.close();
+
+    // Cancel any prefetches in progress.
+    cancelPrefetches();
+
+    cleanupWithLogger(LOG, cache);
+
+    ops.end(op);
+    LOG.info(ops.getSummary(false));
+
+    bufferPool.close();
+    bufferPool = null;
+  }
+
+  /**
+   * Requests optional prefetching of the given block.
+   * The block is prefetched only if we can acquire a free buffer.
+   *
+   * @throws IllegalArgumentException if blockNumber is negative.
+   */
+  @Override
+  public void requestPrefetch(int blockNumber) {
+    checkNotNegative(blockNumber, "blockNumber");
+
+    if (closed) {
+      return;
+    }
+
+    // We initiate a prefetch only if we can acquire a buffer from the shared pool.
+    BufferData data = bufferPool.tryAcquire(blockNumber);
+    if (data == null) {
+      return;
+    }
+
+    // Opportunistic check without locking.
+    if (!data.stateEqualsOneOf(BufferData.State.BLANK)) {
+      // The block is ready or being prefetched/cached.
+      return;
+    }
+
+    synchronized (data) {
+      // Reconfirm state after locking.
+      if (!data.stateEqualsOneOf(BufferData.State.BLANK)) {
+        // The block is ready or being prefetched/cached.
+        return;
+      }
+
+      BlockOperations.Operation op = ops.requestPrefetch(blockNumber);
+      PrefetchTask prefetchTask = new PrefetchTask(data, this, Instant.now());
+      Future<Void> prefetchFuture = futurePool.executeFunction(prefetchTask);
+      data.setPrefetch(prefetchFuture);
+      ops.end(op);
+    }
+  }
+
+  /**
+   * Requests cancellation of any previously issued prefetch requests.
+   */
+  @Override
+  public void cancelPrefetches() {
+    BlockOperations.Operation op = ops.cancelPrefetches();
+
+    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)) {
+        requestCaching(data);
+      }
+    }
+
+    ops.end(op);
+  }
+
+  private void read(BufferData data) throws IOException {
+    synchronized (data) {
+      readBlock(data, false, BufferData.State.BLANK);
+    }
+  }
+
+  private void prefetch(BufferData data, Instant taskQueuedStartTime) throws IOException {
+    synchronized (data) {
+      prefetchingStatistics.executorAcquired(
+          Duration.between(taskQueuedStartTime, Instant.now()));
+      readBlock(
+          data,
+          true,
+          BufferData.State.PREFETCHING,
+          BufferData.State.CACHING);
+    }
+  }
+
+  private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... expectedState)
+      throws IOException {
+
+    if (closed) {
+      return;
+    }
+
+    BlockOperations.Operation op = null;
+    DurationTracker tracker = null;
+
+    synchronized (data) {
+      try {
+        if (data.stateEqualsOneOf(BufferData.State.DONE, BufferData.State.READY)) {
+          // DONE  : Block was released, likely due to caching being disabled on slow perf.
+          // READY : Block was already fetched by another thread. No need to re-read.
+          return;
+        }
+
+        data.throwIfStateIncorrect(expectedState);
+        int blockNumber = data.getBlockNumber();
+
+        // Prefer reading from cache over reading from network.
+        if (cache.containsBlock(blockNumber)) {
+          op = ops.getCached(blockNumber);
+          cache.get(blockNumber, data.getBuffer());
+          data.setReady(expectedState);
+          return;
+        }
+
+        if (isPrefetch) {
+          tracker = prefetchingStatistics.prefetchOperationStarted();
+          op = ops.prefetch(data.getBlockNumber());
+        } else {
+          op = ops.getRead(data.getBlockNumber());
+        }
+
+        long offset = getBlockData().getStartOffset(data.getBlockNumber());
+        int size = getBlockData().getSize(data.getBlockNumber());
+        ByteBuffer buffer = data.getBuffer();
+        buffer.clear();
+        read(buffer, offset, size);
+        buffer.flip();
+        data.setReady(expectedState);
+      } catch (Exception e) {
+        String message = String.format("error during readBlock(%s)", data.getBlockNumber());
+        LOG.error(message, e);
+
+        if (isPrefetch && tracker != null) {
+          tracker.failed();
+        }
+
+        numReadErrors.incrementAndGet();
+        data.setDone();
+        throw e;
+      } finally {
+        if (op != null) {
+          ops.end(op);
+        }
+
+        if (isPrefetch) {
+          prefetchingStatistics.prefetchOperationCompleted();
+          if (tracker != null) {
+            tracker.close();
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Read task that is submitted to the future pool.
+   */
+  private static class PrefetchTask implements Supplier<Void> {
+    private final BufferData data;
+    private final CachingBlockManager blockManager;
+    private final Instant taskQueuedStartTime;
+
+    PrefetchTask(BufferData data, CachingBlockManager blockManager, Instant taskQueuedStartTime) {
+      this.data = data;
+      this.blockManager = blockManager;
+      this.taskQueuedStartTime = taskQueuedStartTime;
+    }
+
+    @Override
+    public Void get() {
+      try {
+        blockManager.prefetch(data, taskQueuedStartTime);
+      } catch (Exception e) {
+        LOG.error("error during prefetch", e);
+      }
+      return null;
+    }
+  }
+
+  private static final BufferData.State[] EXPECTED_STATE_AT_CACHING =
+      new BufferData.State[] {
+          BufferData.State.PREFETCHING, BufferData.State.READY
+      };
+
+  /**
+   * Requests that the given block should be copied to the local cache.
+   * The block must not be accessed by the caller after calling this method
+   * because it will released asynchronously relative to the caller.
+   *
+   * @throws IllegalArgumentException if data is null.
+   */
+  @Override
+  public void requestCaching(BufferData data) {
+    if (closed) {
+      return;
+    }
+
+    if (cachingDisabled.get()) {
+      data.setDone();
+      return;
+    }
+
+    Validate.checkNotNull(data, "data");
+
+    // Opportunistic check without locking.
+    if (!data.stateEqualsOneOf(EXPECTED_STATE_AT_CACHING)) {
+      return;
+    }
+
+    synchronized (data) {
+      // Reconfirm state after locking.
+      if (!data.stateEqualsOneOf(EXPECTED_STATE_AT_CACHING)) {
+        return;
+      }
+
+      if (cache.containsBlock(data.getBlockNumber())) {
+        data.setDone();
+        return;
+      }
+
+      BufferData.State state = data.getState();
+
+      BlockOperations.Operation op = ops.requestCaching(data.getBlockNumber());
+      Future<Void> blockFuture;
+      if (state == BufferData.State.PREFETCHING) {
+        blockFuture = data.getActionFuture();
+      } else {
+        CompletableFuture<Void> cf = new CompletableFuture<>();
+        cf.complete(null);
+        blockFuture = cf;
+      }
+
+      CachePutTask task = new CachePutTask(data, blockFuture, this, Instant.now());
+      Future<Void> actionFuture = futurePool.executeFunction(task);
+      data.setCaching(actionFuture);
+      ops.end(op);
+    }
+  }
+
+  private void addToCacheAndRelease(BufferData data, Future<Void> blockFuture,
+      Instant taskQueuedStartTime) {
+    prefetchingStatistics.executorAcquired(
+        Duration.between(taskQueuedStartTime, Instant.now()));
+
+    if (closed) {
+      return;
+    }
+
+    if (cachingDisabled.get()) {
+      data.setDone();
+      return;
+    }
+
+    try {
+      blockFuture.get(TIMEOUT_MINUTES, TimeUnit.MINUTES);
+      if (data.stateEqualsOneOf(BufferData.State.DONE)) {
+        // There was an error during prefetch.
+        return;
+      }
+    } catch (Exception e) {
+      LOG.error("error waiting on blockFuture: {}", data, e);
+      data.setDone();
+      return;
+    }
+
+    if (cachingDisabled.get()) {
+      data.setDone();
+      return;
+    }
+
+    BlockOperations.Operation op = null;
+
+    synchronized (data) {
+      try {
+        if (data.stateEqualsOneOf(BufferData.State.DONE)) {
+          return;
+        }
+
+        if (cache.containsBlock(data.getBlockNumber())) {
+          data.setDone();
+          return;
+        }
+
+        op = ops.addToCache(data.getBlockNumber());
+        ByteBuffer buffer = data.getBuffer().duplicate();
+        buffer.rewind();
+        cachePut(data.getBlockNumber(), buffer);
+        data.setDone();
+      } catch (Exception e) {
+        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) ops.end(op);
+        if (endOp.duration() > SLOW_CACHING_THRESHOLD) {
+          if (!cachingDisabled.getAndSet(true)) {
+            String message = String.format(
+                "Caching disabled because of slow operation (%.1f sec)", endOp.duration());
+            LOG.warn(message);
+          }
+        }
+      }
+    }
+  }
+
+  protected BlockCache createCache() {
+    return new SingleFilePerBlockCache(prefetchingStatistics);
+  }
+
+  protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException {
+    if (closed) {
+      return;
+    }
+
+    cache.put(blockNumber, buffer);
+  }
+
+  private static class CachePutTask implements Supplier<Void> {
+    private final BufferData data;
+
+    // Block being asynchronously fetched.
+    private final Future<Void> blockFuture;
+
+    // Block manager that manages this block.
+    private final CachingBlockManager blockManager;
+
+    private final Instant taskQueuedStartTime;
+
+    CachePutTask(
+        BufferData data,
+        Future<Void> blockFuture,
+        CachingBlockManager blockManager,
+        Instant taskQueuedStartTime) {
+      this.data = data;
+      this.blockFuture = blockFuture;
+      this.blockManager = blockManager;
+      this.taskQueuedStartTime = taskQueuedStartTime;
+    }
+
+    @Override
+    public Void get() {
+      blockManager.addToCacheAndRelease(data, blockFuture, taskQueuedStartTime);
+      return null;
+    }
+  }
+
+  /**
+   * Number of ByteBuffers available to be acquired.
+   *
+   * @return the number of available buffers.
+   */
+  public int numAvailable() {
+    return bufferPool.numAvailable();
+  }
+
+  /**
+   * Number of caching operations completed.
+   *
+   * @return the number of cached buffers.
+   */
+  public int numCached() {
+    return cache.size();
+  }
+
+  /**
+   * Number of errors encountered when caching.
+   *
+   * @return the number of errors encountered when caching.
+   */
+  public int numCachingErrors() {
+    return numCachingErrors.get();
+  }
+
+  /**
+   * Number of errors encountered when reading.
+   *
+   * @return the number of errors encountered when reading.
+   */
+  public int numReadErrors() {
+    return numReadErrors.get();
+  }
+
+  BufferData getData(int blockNumber) {
+    return bufferPool.tryAcquire(blockNumber);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+
+    sb.append("cache(");
+    sb.append(cache.toString());
+    sb.append("); ");
+
+    sb.append("pool: ");
+    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) {
+
+  }
+}
+

+ 70 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ExecutorServiceFuturePool.java

@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.impl.prefetch;
+
+import java.util.Locale;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.function.Supplier;
+
+/**
+ * A FuturePool implementation backed by a java.util.concurrent.ExecutorService.
+ *
+ * If a piece of work has started, it cannot (currently) be cancelled.
+ *
+ * This class is a simplified version of <code>com.twitter:util-core_2.11</code>
+ * ExecutorServiceFuturePool designed to avoid depending on that Scala library.
+ * One problem with using a Scala library is that many downstream projects
+ * (eg Apache Spark) use Scala, and they might want to use a different version of Scala
+ * from the version that Hadoop chooses to use.
+ *
+ */
+public class ExecutorServiceFuturePool {
+  private ExecutorService executor;
+
+  public ExecutorServiceFuturePool(ExecutorService executor) {
+    this.executor = executor;
+  }
+
+  /**
+   * @param f function to run in future on executor pool
+   * @return future
+   * @throws java.util.concurrent.RejectedExecutionException can be thrown
+   * @throws NullPointerException if f param is null
+   */
+  public Future<Void> executeFunction(final Supplier<Void> f) {
+    return executor.submit(f::get);
+  }
+
+  /**
+   * @param r runnable to run in future on executor pool
+   * @return future
+   * @throws java.util.concurrent.RejectedExecutionException can be thrown
+   * @throws NullPointerException if r param is null
+   */
+  @SuppressWarnings("unchecked")
+  public Future<Void> executeRunnable(final Runnable r) {
+    return (Future<Void>) executor.submit(r::run);
+  }
+
+  public String toString() {
+    return String.format(Locale.ROOT, "ExecutorServiceFuturePool(executor=%s)", executor);
+  }
+}

+ 301 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/FilePosition.java

@@ -0,0 +1,301 @@
+/*
+ * 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.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.
+ *
+ * The file is accessed through an in memory buffer. The absolute position within
+ * the file is the sum of start offset of the buffer within the file and the relative
+ * offset of the current access location within the buffer.
+ *
+ * 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 final class FilePosition {
+
+  /**
+   * Holds block based information about a file.
+   */
+  private BlockData blockData;
+
+  /**
+   * Information about the buffer in use.
+   */
+  private BufferData data;
+
+  /**
+   * Provides access to the underlying file.
+   */
+  private ByteBuffer buffer;
+
+  /**
+   * 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.
+   */
+  private long readStartOffset;
+
+  // Read stats after a seek (mostly for debugging use).
+  private int numSingleByteReads;
+
+  private int numBytesRead;
+
+  private int numBufferReads;
+
+  /**
+   * Constructs an instance of {@link FilePosition}.
+   *
+   * @param fileSize size of the associated file.
+   * @param blockSize size of each block within the file.
+   *
+   * @throws IllegalArgumentException if fileSize is negative.
+   * @throws IllegalArgumentException if blockSize is zero or negative.
+   */
+  public FilePosition(long fileSize, int blockSize) {
+    checkNotNegative(fileSize, "fileSize");
+    if (fileSize == 0) {
+      checkNotNegative(blockSize, "blockSize");
+    } else {
+      checkPositiveInteger(blockSize, "blockSize");
+    }
+
+    this.blockData = new BlockData(fileSize, blockSize);
+
+    // The position is valid only when a valid buffer is associated with this file.
+    this.invalidate();
+  }
+
+  /**
+   * Associates a buffer with this file.
+   *
+   * @param bufferData the buffer associated with this file.
+   * @param startOffset Start offset of the buffer relative to the start of a file.
+   * @param readOffset Offset where reading starts relative to the start of a file.
+   *
+   * @throws IllegalArgumentException if bufferData is null.
+   * @throws IllegalArgumentException if startOffset is negative.
+   * @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) {
+    checkNotNull(bufferData, "bufferData");
+    checkNotNegative(startOffset, "startOffset");
+    checkNotNegative(readOffset, "readOffset");
+    checkWithinRange(
+        readOffset,
+        "readOffset",
+        startOffset,
+        startOffset + bufferData.getBuffer().limit() - 1);
+
+    data = bufferData;
+    buffer = bufferData.getBuffer().duplicate();
+    bufferStartOffset = startOffset;
+    readStartOffset = readOffset;
+    setAbsolute(readOffset);
+
+    resetReadStats();
+  }
+
+  public ByteBuffer buffer() {
+    throwIfInvalidBuffer();
+    return buffer;
+  }
+
+  public BufferData data() {
+    throwIfInvalidBuffer();
+    return data;
+  }
+
+  /**
+   * Gets the current absolute position within this file.
+   *
+   * @return the current absolute position within this file.
+   */
+  public long absolute() {
+    throwIfInvalidBuffer();
+    return bufferStartOffset + relative();
+  }
+
+  /**
+   * If the given {@code pos} lies within the current buffer, updates the current position to
+   * the specified value and returns true; otherwise returns false without changing the position.
+   *
+   * @param pos the absolute position to change the current position to if possible.
+   * @return true if the given current position was updated, false otherwise.
+   */
+  public boolean setAbsolute(long pos) {
+    if (isValid() && isWithinCurrentBuffer(pos)) {
+      int relativePos = (int) (pos - bufferStartOffset);
+      buffer.position(relativePos);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Gets the current position within this file relative to the start of the associated buffer.
+   *
+   * @return the current position within this file relative to the start of the associated buffer.
+   */
+  public int relative() {
+    throwIfInvalidBuffer();
+    return buffer.position();
+  }
+
+  /**
+   * Determines whether the given absolute position lies within the current buffer.
+   *
+   * @param pos the position to check.
+   * @return true if the given absolute position lies within the current buffer, false otherwise.
+   */
+  public boolean isWithinCurrentBuffer(long pos) {
+    throwIfInvalidBuffer();
+    long bufferEndOffset = bufferStartOffset + buffer.limit() - 1;
+    return (pos >= bufferStartOffset) && (pos <= bufferEndOffset);
+  }
+
+  /**
+   * Gets the id of the current block.
+   *
+   * @return the id of the current block.
+   */
+  public int blockNumber() {
+    throwIfInvalidBuffer();
+    return blockData.getBlockNumber(bufferStartOffset);
+  }
+
+  /**
+   * Determines whether the current block is the last block in this file.
+   *
+   * @return true if the current block is the last block in this file, false otherwise.
+   */
+  public boolean isLastBlock() {
+    return blockData.isLastBlock(blockNumber());
+  }
+
+  /**
+   * Determines if the current position is valid.
+   *
+   * @return true if the current position is valid, false otherwise.
+   */
+  public boolean isValid() {
+    return buffer != null;
+  }
+
+  /**
+   * Marks the current position as invalid.
+   */
+  public void invalidate() {
+    buffer = null;
+    bufferStartOffset = -1;
+    data = null;
+  }
+
+  /**
+   * Gets the start of the current block's absolute offset.
+   *
+   * @return the start of the current block's absolute offset.
+   */
+  public long bufferStartOffset() {
+    throwIfInvalidBuffer();
+    return bufferStartOffset;
+  }
+
+  /**
+   * Determines whether the current buffer has been fully read.
+   *
+   * @return true if the current buffer has been fully read, false otherwise.
+   */
+  public boolean bufferFullyRead() {
+    throwIfInvalidBuffer();
+    return (bufferStartOffset == readStartOffset)
+        && (relative() == buffer.limit())
+        && (numBytesRead == buffer.limit());
+  }
+
+  public void incrementBytesRead(int n) {
+    numBytesRead += n;
+    if (n == 1) {
+      numSingleByteReads++;
+    } else {
+      numBufferReads++;
+    }
+  }
+
+  public int numBytesRead() {
+    return numBytesRead;
+  }
+
+  public int numSingleByteReads() {
+    return numSingleByteReads;
+  }
+
+  public int numBufferReads() {
+    return numBufferReads;
+  }
+
+  private void resetReadStats() {
+    numBytesRead = 0;
+    numSingleByteReads = 0;
+    numBufferReads = 0;
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    if (buffer == null) {
+      sb.append("currentBuffer = null");
+    } else {
+      int pos = buffer.position();
+      int val;
+      if (pos >= buffer.limit()) {
+        val = -1;
+      } else {
+        val = buffer.get(pos);
+      }
+      String currentBufferState =
+          String.format("%d at pos: %d, lim: %d", val, pos, buffer.limit());
+      sb.append(String.format(
+          "block: %d, pos: %d (CBuf: %s)%n",
+          blockNumber(), absolute(),
+          currentBufferState));
+      sb.append("\n");
+    }
+    return sb.toString();
+  }
+
+  private void throwIfInvalidBuffer() {
+    checkState(buffer != null, "'buffer' must not be null");
+  }
+}

+ 67 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/PrefetchingStatistics.java

@@ -0,0 +1,67 @@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements. See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership. The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License. You may obtain a copy of the License at
+  *
+  *   http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing,
+  * software distributed under the License is distributed on an
+  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  * KIND, either express or implied. See the License for the
+  * specific language governing permissions and limitations
+  * under the License.
+  */
+
+package org.apache.hadoop.fs.impl.prefetch;
+
+import java.time.Duration;
+
+import org.apache.hadoop.fs.statistics.DurationTracker;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+public interface PrefetchingStatistics extends IOStatisticsSource {
+
+  /**
+   * A prefetch operation has started.
+   * @return duration tracker
+   */
+  DurationTracker prefetchOperationStarted();
+
+  /**
+   * A block has been saved to the file cache.
+   */
+  void blockAddedToFileCache();
+
+  /**
+   * A block has been removed from the file cache.
+   */
+  void blockRemovedFromFileCache();
+
+  /**
+   * A prefetch operation has completed.
+   */
+  void prefetchOperationCompleted();
+
+  /**
+   * An executor has been acquired, either for prefetching or caching.
+   * @param timeInQueue time taken to acquire an executor.
+   */
+  void executorAcquired(Duration timeInQueue);
+
+  /**
+   * A new buffer has been added to the buffer pool.
+   * @param size size of the new buffer
+   */
+  void memoryAllocated(int size);
+
+  /**
+   * Previously allocated memory has been freed.
+   * @param size size of memory freed.
+   */
+  void memoryFreed(int size);
+}

+ 71 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ResourcePool.java

@@ -0,0 +1,71 @@
+/*
+ * 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.io.Closeable;
+
+/**
+ * Manages a fixed pool of resources.
+ *
+ * Avoids creating a new resource if a previously created instance is already available.
+ */
+public abstract class ResourcePool<T> implements Closeable {
+
+  /**
+   * Acquires a resource blocking if necessary until one becomes available.
+   *
+   * @return the acquired resource instance.
+   */
+  public abstract T acquire();
+
+  /**
+   * Acquires a resource blocking if one is immediately available. Otherwise returns null.
+
+   * @return the acquired resource instance (if immediately available) or null.
+   */
+  public abstract T tryAcquire();
+
+  /**
+   * Releases a previously acquired resource.
+   *
+   * @param item the resource to release.
+   */
+  public abstract void release(T item);
+
+  @Override
+  public void close() {
+  }
+
+  /**
+   * Derived classes may implement a way to cleanup each item.
+   *
+   * @param item the resource to close.
+   */
+  protected void close(T item) {
+    // Do nothing in this class. Allow overriding classes to take any cleanup action.
+  }
+
+  /**
+   * Derived classes must implement a way to create an instance of a resource.
+   *
+   * @return the created instance.
+   */
+  protected abstract T createNew();
+}

+ 93 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Retryer.java

@@ -0,0 +1,93 @@
+/*
+ * 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 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. */
+  private int maxDelay;
+
+  /* Per retry delay (in ms). */
+  private int perRetryDelay;
+
+  /**
+   * The time interval (in ms) at which status update would be made.
+   */
+  private int statusUpdateInterval;
+
+  /* Current delay. */
+  private int delay;
+
+  /**
+   * Initializes a new instance of the {@code Retryer} class.
+   *
+   * @param perRetryDelay per retry delay (in ms).
+   * @param maxDelay maximum amount of delay (in ms) before retry fails.
+   * @param statusUpdateInterval time interval (in ms) at which status update would be made.
+   *
+   * @throws IllegalArgumentException if perRetryDelay is zero or negative.
+   * @throws IllegalArgumentException if maxDelay is less than or equal to perRetryDelay.
+   * @throws IllegalArgumentException if statusUpdateInterval is zero or negative.
+   */
+  public Retryer(int perRetryDelay, int maxDelay, int statusUpdateInterval) {
+    checkPositiveInteger(perRetryDelay, "perRetryDelay");
+    checkGreater(maxDelay, "maxDelay", perRetryDelay, "perRetryDelay");
+    checkPositiveInteger(statusUpdateInterval, "statusUpdateInterval");
+
+    this.perRetryDelay = perRetryDelay;
+    this.maxDelay = maxDelay;
+    this.statusUpdateInterval = statusUpdateInterval;
+  }
+
+  /**
+   * Returns true if retrying should continue, false otherwise.
+   *
+   * @return true if the caller should retry, false otherwise.
+   */
+  public boolean continueRetry() {
+    if (this.delay >= this.maxDelay) {
+      return false;
+    }
+
+    try {
+      Thread.sleep(this.perRetryDelay);
+    } catch (InterruptedException e) {
+      // Ignore the exception as required by the semantic of this class;
+    }
+
+    this.delay += this.perRetryDelay;
+    return true;
+  }
+
+  /**
+   * Returns true if status update interval has been reached.
+   *
+   * @return true if status update interval has been reached.
+   */
+  public boolean updateStatus() {
+    return (this.delay > 0) && this.delay % this.statusUpdateInterval == 0;
+  }
+}

+ 354 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java

@@ -0,0 +1,354 @@
+/*
+ * 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.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.nio.file.attribute.FileAttribute;
+import java.nio.file.attribute.PosixFilePermission;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+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.
+ * Each cache block is stored on the local disk as a separate file.
+ */
+public class SingleFilePerBlockCache implements BlockCache {
+  private static final Logger LOG = LoggerFactory.getLogger(SingleFilePerBlockCache.class);
+
+  /**
+   * 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.
+   */
+  private int numGets = 0;
+
+  private boolean closed;
+
+  private final PrefetchingStatistics prefetchingStatistics;
+
+  /**
+   * 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;
+    private final long checksum;
+
+    Entry(int blockNumber, Path path, int size, long checksum) {
+      this.blockNumber = blockNumber;
+      this.path = path;
+      this.size = size;
+      this.checksum = checksum;
+    }
+
+    @Override
+    public String toString() {
+      return String.format(
+          "([%03d] %s: size = %d, checksum = %d)",
+          blockNumber, path, size, checksum);
+    }
+  }
+
+  /**
+   * Constructs an instance of a {@code SingleFilePerBlockCache}.
+   *
+   * @param prefetchingStatistics statistics for this stream.
+   */
+  public SingleFilePerBlockCache(PrefetchingStatistics prefetchingStatistics) {
+    this.prefetchingStatistics = requireNonNull(prefetchingStatistics);
+  }
+
+  /**
+   * Indicates whether the given block is in this cache.
+   */
+  @Override
+  public boolean containsBlock(int blockNumber) {
+    return blocks.containsKey(blockNumber);
+  }
+
+  /**
+   * Gets the blocks in this cache.
+   */
+  @Override
+  public Iterable<Integer> blocks() {
+    return Collections.unmodifiableList(new ArrayList<>(blocks.keySet()));
+  }
+
+  /**
+   * Gets the number of blocks in this cache.
+   */
+  @Override
+  public int size() {
+    return blocks.size();
+  }
+
+  /**
+   * Gets the block having the given {@code blockNumber}.
+   *
+   * @throws IllegalArgumentException if buffer is null.
+   */
+  @Override
+  public void get(int blockNumber, ByteBuffer buffer) throws IOException {
+    if (closed) {
+      return;
+    }
+
+    checkNotNull(buffer, "buffer");
+
+    Entry entry = getEntry(blockNumber);
+    buffer.clear();
+    readFile(entry.path, buffer);
+    buffer.rewind();
+
+    validateEntry(entry, buffer);
+  }
+
+  protected int readFile(Path path, ByteBuffer buffer) throws IOException {
+    int numBytesRead = 0;
+    int numBytes;
+    FileChannel channel = FileChannel.open(path, StandardOpenOption.READ);
+    while ((numBytes = channel.read(buffer)) > 0) {
+      numBytesRead += numBytes;
+    }
+    buffer.limit(buffer.position());
+    channel.close();
+    return numBytesRead;
+  }
+
+  private Entry getEntry(int blockNumber) {
+    Validate.checkNotNegative(blockNumber, "blockNumber");
+
+    Entry entry = blocks.get(blockNumber);
+    if (entry == null) {
+      throw new IllegalStateException(String.format("block %d not found in cache", blockNumber));
+    }
+    numGets++;
+    return entry;
+  }
+
+  /**
+   * Puts the given block in this cache.
+   *
+   * @throws IllegalArgumentException if buffer is null.
+   * @throws IllegalArgumentException if buffer.limit() is zero or negative.
+   */
+  @Override
+  public void put(int blockNumber, ByteBuffer buffer) throws IOException {
+    if (closed) {
+      return;
+    }
+
+    checkNotNull(buffer, "buffer");
+
+    if (blocks.containsKey(blockNumber)) {
+      Entry entry = blocks.get(blockNumber);
+      validateEntry(entry, buffer);
+      return;
+    }
+
+    Validate.checkPositiveInteger(buffer.limit(), "buffer.limit()");
+
+    Path blockFilePath = getCacheFilePath();
+    long size = Files.size(blockFilePath);
+    if (size != 0) {
+      String message =
+          String.format("[%d] temp file already has data. %s (%d)",
+          blockNumber, blockFilePath, size);
+      throw new IllegalStateException(message);
+    }
+
+    writeFile(blockFilePath, buffer);
+    prefetchingStatistics.blockAddedToFileCache();
+    long checksum = BufferData.getChecksum(buffer);
+    Entry entry = new Entry(blockNumber, blockFilePath, buffer.limit(), checksum);
+    blocks.put(blockNumber, entry);
+  }
+
+  private static final Set<? extends OpenOption> CREATE_OPTIONS =
+      EnumSet.of(StandardOpenOption.WRITE,
+          StandardOpenOption.CREATE,
+          StandardOpenOption.TRUNCATE_EXISTING);
+
+  protected void writeFile(Path path, ByteBuffer buffer) throws IOException {
+    buffer.rewind();
+    WritableByteChannel writeChannel = Files.newByteChannel(path, CREATE_OPTIONS);
+    while (buffer.hasRemaining()) {
+      writeChannel.write(buffer);
+    }
+    writeChannel.close();
+  }
+
+  protected Path getCacheFilePath() throws IOException {
+    return getTempFilePath();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closed) {
+      return;
+    }
+
+    closed = true;
+
+    LOG.info(getStats());
+    int numFilesDeleted = 0;
+
+    for (Entry entry : blocks.values()) {
+      try {
+        Files.deleteIfExists(entry.path);
+        prefetchingStatistics.blockRemovedFromFileCache();
+        numFilesDeleted++;
+      } catch (IOException e) {
+        // Ignore while closing so that we can delete as many cache files as possible.
+      }
+    }
+
+    if (numFilesDeleted > 0) {
+      LOG.info("Deleted {} cache files", numFilesDeleted);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("stats: ");
+    sb.append(getStats());
+    sb.append(", blocks:[");
+    sb.append(getIntList(blocks()));
+    sb.append("]");
+    return sb.toString();
+  }
+
+  private void validateEntry(Entry entry, ByteBuffer buffer) {
+    if (entry.size != buffer.limit()) {
+      String message = String.format(
+          "[%d] entry.size(%d) != buffer.limit(%d)",
+          entry.blockNumber, entry.size, buffer.limit());
+      throw new IllegalStateException(message);
+    }
+
+    long checksum = BufferData.getChecksum(buffer);
+    if (entry.checksum != checksum) {
+      String message = String.format(
+          "[%d] entry.checksum(%d) != buffer checksum(%d)",
+          entry.blockNumber, entry.checksum, checksum);
+      throw new IllegalStateException(message);
+    }
+  }
+
+  /**
+   * Produces a human readable list of blocks for the purpose of logging.
+   * This method minimizes the length of returned list by converting
+   * a contiguous list of blocks into a range.
+   * for example,
+   * 1, 3, 4, 5, 6, 8 becomes 1, 3~6, 8
+   */
+  private String getIntList(Iterable<Integer> nums) {
+    List<String> numList = new ArrayList<>();
+    List<Integer> numbers = new ArrayList<Integer>();
+    for (Integer n : nums) {
+      numbers.add(n);
+    }
+    Collections.sort(numbers);
+
+    int index = 0;
+    while (index < numbers.size()) {
+      int start = numbers.get(index);
+      int prev = start;
+      int end = start;
+      while ((++index < numbers.size()) && ((end = numbers.get(index)) == prev + 1)) {
+        prev = end;
+      }
+
+      if (start == prev) {
+        numList.add(Integer.toString(start));
+      } else {
+        numList.add(String.format("%d~%d", start, prev));
+      }
+    }
+
+    return String.join(", ", numList);
+  }
+
+  private String getStats() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(String.format(
+        "#entries = %d, #gets = %d",
+        blocks.size(), numGets));
+    return sb.toString();
+  }
+
+  private static final String CACHE_FILE_PREFIX = "fs-cache-";
+
+  public static boolean isCacheSpaceAvailable(long fileSize) {
+    try {
+      Path cacheFilePath = getTempFilePath();
+      long freeSpace = new File(cacheFilePath.toString()).getUsableSpace();
+      LOG.info("fileSize = {}, freeSpace = {}", fileSize, freeSpace);
+      Files.deleteIfExists(cacheFilePath);
+      return fileSize < freeSpace;
+    } catch (IOException e) {
+      LOG.error("isCacheSpaceAvailable", e);
+      return false;
+    }
+  }
+
+  // The suffix (file extension) of each serialized index file.
+  private static final String BINARY_FILE_SUFFIX = ".bin";
+
+  // File attributes attached to any intermediate temporary file created during index creation.
+  private static final FileAttribute<Set<PosixFilePermission>> TEMP_FILE_ATTRS =
+      PosixFilePermissions.asFileAttribute(EnumSet.of(PosixFilePermission.OWNER_READ,
+          PosixFilePermission.OWNER_WRITE));
+
+  private static Path getTempFilePath() throws IOException {
+    return Files.createTempFile(
+        CACHE_FILE_PREFIX,
+        BINARY_FILE_SUFFIX,
+        TEMP_FILE_ATTRS
+    );
+  }
+}

+ 399 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Validate.java

@@ -0,0 +1,399 @@
+/*
+ * 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.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() {
+  }
+
+  /**
+   * 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.
+   */
+  public static void checkNotNull(Object obj, String argName) {
+    checkArgument(obj != null, "'%s' must not be null.", argName);
+  }
+
+  /**
+   * 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.
+   */
+  public static void checkPositiveInteger(long value, String argName) {
+    checkArgument(value > 0, "'%s' must be a positive integer.", argName);
+  }
+
+  /**
+   * 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.
+   */
+  public static void checkNotNegative(long value, String argName) {
+    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.
+   */
+  public static void checkRequired(boolean isPresent, String argName) {
+    checkArgument(isPresent, "'%s' is required.", argName);
+  }
+
+  /**
+   * 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.
+   */
+  public static void checkValid(boolean isValid, String argName) {
+    checkArgument(isValid, "'%s' is invalid.", argName);
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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) {
+    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) {
+    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) {
+    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) {
+    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) {
+    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) {
+    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) {
+    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.
+   * @param argName the name of the argument being validated.
+   */
+  public static <T> void checkNotNullAndNumberOfElements(
+      Collection<T> collection, int numElements, String argName) {
+    checkNotNull(collection, argName);
+    checkArgument(
+        collection.size() == numElements,
+        "Number of elements in '%s' must be exactly %s, %s given.",
+        argName,
+        numElements,
+        collection.size()
+    );
+  }
+
+  /**
+   * 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.
+   * @param value2Name the name of the second argument.
+   */
+  public static void checkValuesEqual(
+      long value1,
+      String value1Name,
+      long value2,
+      String value2Name) {
+    checkArgument(
+        value1 == value2,
+        "'%s' (%s) must equal '%s' (%s).",
+        value1Name,
+        value1,
+        value2Name,
+        value2);
+  }
+
+  /**
+   * 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.
+   * @param value2Name the name of the second argument.
+   */
+  public static void checkIntegerMultiple(
+      long value1,
+      String value1Name,
+      long value2,
+      String value2Name) {
+    checkArgument(
+        (value1 % value2) == 0,
+        "'%s' (%s) must be an integer multiple of '%s' (%s).",
+        value1Name,
+        value1,
+        value2Name,
+        value2);
+  }
+
+  /**
+   * 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.
+   * @param value2Name the name of the second argument.
+   */
+  public static void checkGreater(
+      long value1,
+      String value1Name,
+      long value2,
+      String value2Name) {
+    checkArgument(
+        value1 > value2,
+        "'%s' (%s) must be greater than '%s' (%s).",
+        value1Name,
+        value1,
+        value2Name,
+        value2);
+  }
+
+  /**
+   * 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.
+   * @param value2Name the name of the second argument.
+   */
+  public static void checkGreaterOrEqual(
+      long value1,
+      String value1Name,
+      long value2,
+      String value2Name) {
+    checkArgument(
+        value1 >= value2,
+        "'%s' (%s) must be greater than or equal to '%s' (%s).",
+        value1Name,
+        value1,
+        value2Name,
+        value2);
+  }
+
+  /**
+   * 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.
+   * @param value2Name the name of the second argument.
+   */
+  public static void checkLessOrEqual(
+      long value1,
+      String value1Name,
+      long value2,
+      String value2Name) {
+    checkArgument(
+        value1 <= value2,
+        "'%s' (%s) must be less than or equal to '%s' (%s).",
+        value1Name,
+        value1,
+        value2Name,
+        value2);
+  }
+
+  /**
+   * 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.
+   * @param maxValueInclusive inclusive upper limit for the value.
+   */
+  public static void checkWithinRange(
+      long value,
+      String valueName,
+      long minValueInclusive,
+      long maxValueInclusive) {
+    checkArgument(
+        (value >= minValueInclusive) && (value <= maxValueInclusive),
+        "'%s' (%s) must be within the range [%s, %s].",
+        valueName,
+        value,
+        minValueInclusive,
+        maxValueInclusive);
+  }
+
+  /**
+   * 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.
+   * @param maxValueInclusive inclusive upper limit for the value.
+   */
+  public static void checkWithinRange(
+      double value,
+      String valueName,
+      double minValueInclusive,
+      double maxValueInclusive) {
+    checkArgument(
+        (value >= minValueInclusive) && (value <= maxValueInclusive),
+        "'%s' (%s) must be within the range [%s, %s].",
+        valueName,
+        value,
+        minValueInclusive,
+        maxValueInclusive);
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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.
+   */
+  public static void checkPathExistsAsDir(Path path, String argName) {
+    checkPathExists(path, argName);
+    checkArgument(
+        Files.isDirectory(path),
+        "Path %s (%s) must point to a directory.",
+        argName,
+        path);
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /**
+   * 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) {
+    checkArgument(
+        arraySize > 0,
+        "'%s' must have at least one element.",
+        argName);
+  }
+}

+ 28 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/package-info.java

@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/**
+ * block caching for use in object store clients.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.impl.prefetch;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 40 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java

@@ -415,6 +415,46 @@ public final class StreamStatisticNames {
   public static final String BLOCKS_RELEASED
       = "blocks_released";
 
+  /**
+   * Total number of prefetching operations executed.
+   */
+  public static final String STREAM_READ_PREFETCH_OPERATIONS
+      = "stream_read_prefetch_operations";
+
+  /**
+   * Total number of block in disk cache.
+   */
+  public static final String STREAM_READ_BLOCKS_IN_FILE_CACHE
+      = "stream_read_blocks_in_cache";
+
+  /**
+   * Total number of active prefetch operations.
+   */
+  public static final String STREAM_READ_ACTIVE_PREFETCH_OPERATIONS
+      = "stream_read_active_prefetch_operations";
+
+  /**
+   * Total bytes of memory in use by this input stream.
+   */
+  public static final String STREAM_READ_ACTIVE_MEMORY_IN_USE
+      = "stream_read_active_memory_in_use";
+
+  /**
+   * count/duration of reading a remote block.
+   *
+   * Value: {@value}.
+   */
+  public static final String STREAM_READ_REMOTE_BLOCK_READ
+      = "stream_read_block_read";
+
+  /**
+   * count/duration of acquiring a buffer and reading to it.
+   *
+   * Value: {@value}.
+   */
+  public static final String STREAM_READ_BLOCK_ACQUIRE_AND_READ
+      = "stream_read_block_acquire_read";
+
   private StreamStatisticNames() {
   }
 

+ 63 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/ExceptionAsserts.java

@@ -0,0 +1,63 @@
+/*
+ * 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 org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public final class ExceptionAsserts {
+
+  private ExceptionAsserts() {
+  }
+
+
+  /**
+   * Asserts that the given code throws an exception of the given type
+   * and that the exception message contains the given sub-message.
+   *
+   * Usage:
+   *
+   * ExceptionAsserts.assertThrows(
+   *   IllegalArgumentException.class,
+   *   "'nullArg' must not be null",
+   *   () -> Preconditions.checkNotNull(null, "nullArg"));
+   *
+   * Note: JUnit 5 has similar functionality but it will be a long time before
+   * we move to that framework because of significant differences and lack of
+   * backward compatibility for some JUnit rules.
+   */
+  public static <E extends Exception> void assertThrows(
+      Class<E> expectedExceptionClass,
+      String partialMessage,
+      LambdaTestUtils.VoidCallable code) throws Exception {
+
+    intercept(expectedExceptionClass, partialMessage, code);
+
+  }
+
+  public static <E extends Exception> void assertThrows(
+      Class<E> expectedExceptionClass,
+      LambdaTestUtils.VoidCallable code) throws Exception {
+
+    intercept(expectedExceptionClass, code);
+
+  }
+}

+ 71 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/SampleDataForTests.java

@@ -0,0 +1,71 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Frequently used test data items.
+ */
+public final class 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[] 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[] 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[] 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 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> EMPTY_LIST = new ArrayList<Object>();
+
+  public static final List<Object> VALID_LIST = Arrays.asList(new Object[1]);
+}

+ 98 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockCache.java

@@ -0,0 +1,98 @@
+/*
+ * 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.nio.ByteBuffer;
+
+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;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+
+public class TestBlockCache extends AbstractHadoopTestBase {
+
+  private static final int BUFFER_SIZE = 16;
+
+  @Test
+  public void testArgChecks() throws Exception {
+    // Should not throw.
+    BlockCache cache =
+        new SingleFilePerBlockCache(EmptyPrefetchingStatistics.getInstance());
+
+    ByteBuffer buffer = ByteBuffer.allocate(16);
+
+    // Verify it throws correctly.
+    intercept(IllegalArgumentException.class, "'buffer' must not be null",
+        () -> cache.put(42, null));
+
+
+    intercept(NullPointerException.class, null,
+        () -> new SingleFilePerBlockCache(null));
+
+  }
+
+
+  @Test
+  public void testPutAndGet() throws Exception {
+    BlockCache cache =
+        new SingleFilePerBlockCache(EmptyPrefetchingStatistics.getInstance());
+
+    ByteBuffer buffer1 = ByteBuffer.allocate(BUFFER_SIZE);
+    for (byte i = 0; i < BUFFER_SIZE; i++) {
+      buffer1.put(i);
+    }
+
+    assertEquals(0, cache.size());
+    assertFalse(cache.containsBlock(0));
+    cache.put(0, buffer1);
+    assertEquals(1, cache.size());
+    assertTrue(cache.containsBlock(0));
+    ByteBuffer buffer2 = ByteBuffer.allocate(BUFFER_SIZE);
+    cache.get(0, buffer2);
+    assertNotSame(buffer1, buffer2);
+    assertBuffersEqual(buffer1, buffer2);
+
+    assertEquals(1, cache.size());
+    assertFalse(cache.containsBlock(1));
+    cache.put(1, buffer1);
+    assertEquals(2, cache.size());
+    assertTrue(cache.containsBlock(1));
+    ByteBuffer buffer3 = ByteBuffer.allocate(BUFFER_SIZE);
+    cache.get(1, buffer3);
+    assertNotSame(buffer1, buffer3);
+    assertBuffersEqual(buffer1, buffer3);
+  }
+
+  private void assertBuffersEqual(ByteBuffer buffer1, ByteBuffer buffer2) {
+    assertNotNull(buffer1);
+    assertNotNull(buffer2);
+    assertEquals(buffer1.limit(), buffer2.limit());
+    assertEquals(BUFFER_SIZE, buffer1.limit());
+    for (int i = 0; i < BUFFER_SIZE; i++) {
+      assertEquals(buffer1.get(i), buffer2.get(i));
+    }
+  }
+}

+ 159 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockData.java

@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.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;
+
+public class TestBlockData extends AbstractHadoopTestBase {
+
+  @Test
+  public void testArgChecks() throws Exception {
+    // Should not throw.
+    new BlockData(10, 5);
+    new BlockData(5, 10);
+    new BlockData(0, 10);
+
+    // Verify it throws correctly.
+
+
+    intercept(IllegalArgumentException.class, "'fileSize' must not be negative",
+        () -> new BlockData(-1, 2));
+
+    intercept(IllegalArgumentException.class,
+        "'blockSize' must be a positive integer",
+        () -> new BlockData(10, 0));
+
+    intercept(IllegalArgumentException.class,
+        "'blockSize' must be a positive integer",
+        () -> new BlockData(10, -2));
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' (-1) must be within the range [0, 3]",
+        () -> new BlockData(10, 3).isLastBlock(
+            -1));
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' (11) must be within the range [0, 3]",
+        () -> new BlockData(10, 3).isLastBlock(
+            11));
+
+  }
+
+  @Test
+  public void testComputedFields() throws Exception {
+    testComputedFieldsHelper(0, 10);
+    testComputedFieldsHelper(1, 10);
+    testComputedFieldsHelper(10, 1);
+    testComputedFieldsHelper(10, 2);
+    testComputedFieldsHelper(10, 3);
+  }
+
+  private void testComputedFieldsHelper(long fileSize, int blockSize)
+      throws Exception {
+    BlockData bd = new BlockData(fileSize, blockSize);
+
+    if (fileSize == 0) {
+      assertFalse(bd.isLastBlock(0));
+      assertFalse(bd.isLastBlock(1));
+      assertFalse(bd.isValidOffset(0));
+      assertEquals(0, bd.getSize(0));
+      assertEquals("", bd.getStateString());
+
+      ExceptionAsserts.assertThrows(
+          IllegalArgumentException.class,
+          "'offset' (0) must be within the range [0, -1]",
+          () -> bd.getBlockNumber(0));
+
+      ExceptionAsserts.assertThrows(
+          IllegalArgumentException.class,
+          "'blockNumber' (0) must be within the range [0, -1]",
+          () -> bd.getStartOffset(0));
+
+      ExceptionAsserts.assertThrows(
+          IllegalArgumentException.class,
+          "'offset' (0) must be within the range [0, -1]",
+          () -> bd.getRelativeOffset(0, 0));
+
+      ExceptionAsserts.assertThrows(
+          IllegalArgumentException.class,
+          "'blockNumber' (0) must be within the range [0, -1]",
+          () -> bd.getState(0));
+
+      ExceptionAsserts.assertThrows(
+          IllegalArgumentException.class,
+          "'blockNumber' (0) must be within the range [0, -1]",
+          () -> bd.setState(0, BlockData.State.READY));
+
+      return;
+    }
+
+    assertEquals(fileSize, bd.getFileSize());
+    assertEquals(blockSize, bd.getBlockSize());
+
+    int expectedNumBlocks = (int) (fileSize / blockSize);
+    if (fileSize % blockSize > 0) {
+      expectedNumBlocks++;
+    }
+    assertEquals(expectedNumBlocks, bd.getNumBlocks());
+
+    int lastBlockNumber = expectedNumBlocks - 1;
+    for (int b = 0; b < lastBlockNumber; b++) {
+      assertFalse(bd.isLastBlock(b));
+      assertEquals(blockSize, bd.getSize(b));
+    }
+    assertTrue(bd.isLastBlock(lastBlockNumber));
+    int lastBlockSize = (int) (fileSize - blockSize * (expectedNumBlocks - 1));
+    assertEquals(lastBlockSize, bd.getSize(lastBlockNumber));
+
+    // Offset related methods.
+    for (long offset = 0; offset < fileSize; offset++) {
+      int expectedBlockNumber = (int) (offset / blockSize);
+      assertEquals(expectedBlockNumber, bd.getBlockNumber(offset));
+
+      for (int b = 0; b < expectedNumBlocks - 1; b++) {
+        long expectedStartOffset = b * blockSize;
+        assertEquals(expectedStartOffset, bd.getStartOffset(b));
+
+        int expectedRelativeOffset = (int) (offset - expectedStartOffset);
+        assertEquals(expectedRelativeOffset, bd.getRelativeOffset(b, offset));
+      }
+    }
+
+
+    // State methods.
+    for (int b = 0; b < expectedNumBlocks; b++) {
+      assertEquals(b * blockSize, bd.getStartOffset(b));
+      assertEquals(BlockData.State.NOT_READY, bd.getState(b));
+      bd.setState(b, BlockData.State.QUEUED);
+      assertEquals(BlockData.State.QUEUED, bd.getState(b));
+      bd.setState(b, BlockData.State.READY);
+      assertEquals(BlockData.State.READY, bd.getState(b));
+      bd.setState(b, BlockData.State.CACHED);
+      assertEquals(BlockData.State.CACHED, bd.getState(b));
+    }
+  }
+}

+ 106 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockOperations.java

@@ -0,0 +1,106 @@
+/*
+ * 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.lang.reflect.Method;
+
+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 {
+
+  @Test
+  public void testArgChecks() throws Exception {
+    // Should not throw.
+    BlockOperations ops = new BlockOperations();
+
+    // Verify it throws correctly.
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> ops.getPrefetched(-1));
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> ops.getCached(-1));
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> ops.getRead(-1));
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> ops.release(-1));
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> ops.requestPrefetch(-1));
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> ops.requestCaching(-1));
+
+  }
+
+  @Test
+  public void testGetSummary() throws Exception {
+    verifySummary("getPrefetched", "GP");
+    verifySummary("getCached", "GC");
+    verifySummary("getRead", "GR");
+    verifySummary("release", "RL");
+    verifySummary("requestPrefetch", "RP");
+    verifySummary("prefetch", "PF");
+    verifySummary("requestCaching", "RC");
+    verifySummary("addToCache", "C+");
+
+    verifySummaryNoArg("cancelPrefetches", "CP");
+    verifySummaryNoArg("close", "CX");
+  }
+
+  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);
+    ops.end(op);
+    String summary = ops.getSummary(false);
+    String opSummary = String.format("%s(%d)", shortName, blockNumber);
+    String expectedSummary = String.format("%s;E%s;", opSummary, opSummary);
+    assertTrue(summary.startsWith(expectedSummary));
+  }
+
+  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);
+    ops.end(op);
+    String summary = ops.getSummary(false);
+    String expectedSummary = String.format("%s;E%s;", shortName, shortName);
+    assertTrue(summary.startsWith(expectedSummary));
+  }
+}

+ 147 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBoundedResourcePool.java

@@ -0,0 +1,147 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.Set;
+
+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;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+public class TestBoundedResourcePool extends AbstractHadoopTestBase {
+
+  static class BufferPool extends BoundedResourcePool<ByteBuffer> {
+
+    BufferPool(int size) {
+      super(size);
+    }
+
+    @Override
+    protected ByteBuffer createNew() {
+      return ByteBuffer.allocate(10);
+    }
+  }
+
+  @Test
+  public void testArgChecks() throws Exception {
+
+    // Should not throw.
+    BufferPool pool = new BufferPool(5);
+
+    // Verify it throws correctly.
+
+    intercept(IllegalArgumentException.class,
+        "'size' must be a positive integer",
+        () -> new BufferPool(-1));
+
+    intercept(IllegalArgumentException.class,
+        "'size' must be a positive integer",
+        () -> new BufferPool(0));
+
+    intercept(IllegalArgumentException.class, "'item' must not be null",
+        () -> pool.release(null));
+
+    intercept(IllegalArgumentException.class,
+        "This item is not a part of this pool",
+        () -> pool.release(ByteBuffer.allocate(4)));
+
+  }
+
+  @Test
+  public void testAcquireReleaseSingle() {
+    final int numBuffers = 5;
+    BufferPool pool = new BufferPool(numBuffers);
+
+    assertEquals(0, pool.numCreated());
+    assertEquals(numBuffers, pool.numAvailable());
+
+    ByteBuffer buffer1 = pool.acquire();
+    assertNotNull(buffer1);
+    assertEquals(1, pool.numCreated());
+    assertEquals(numBuffers - 1, pool.numAvailable());
+
+    // Release and immediately reacquire => should not end up creating new buffer.
+    pool.release(buffer1);
+    assertEquals(1, pool.numCreated());
+
+    ByteBuffer buffer2 = pool.acquire();
+    assertNotNull(buffer2);
+    assertSame(buffer1, buffer2);
+    assertEquals(1, pool.numCreated());
+  }
+
+  @Test
+  public void testAcquireReleaseMultiple() {
+    final int numBuffers = 5;
+    BufferPool pool = new BufferPool(numBuffers);
+    Set<ByteBuffer> buffers =
+        Collections.newSetFromMap(new IdentityHashMap<ByteBuffer, Boolean>());
+
+    assertEquals(0, pool.numCreated());
+
+    // Acquire all one by one.
+    for (int i = 0; i < numBuffers; i++) {
+      assertEquals(numBuffers - i, pool.numAvailable());
+      ByteBuffer buffer = pool.acquire();
+      assertNotNull(buffer);
+      assertFalse(buffers.contains(buffer));
+      buffers.add(buffer);
+      assertEquals(i + 1, pool.numCreated());
+    }
+
+    assertEquals(numBuffers, pool.numCreated());
+    assertEquals(0, pool.numAvailable());
+
+    int releaseCount = 0;
+
+    // Release all one by one.
+    for (ByteBuffer buffer : buffers) {
+      assertEquals(releaseCount, pool.numAvailable());
+      releaseCount++;
+      pool.release(buffer);
+      assertEquals(releaseCount, pool.numAvailable());
+
+      // Releasing the same buffer again should not have any ill effect.
+      pool.release(buffer);
+      assertEquals(releaseCount, pool.numAvailable());
+      pool.release(buffer);
+      assertEquals(releaseCount, pool.numAvailable());
+    }
+
+    // Acquire all one by one again to ensure that they are the same ones we got earlier.
+    for (int i = 0; i < numBuffers; i++) {
+      ByteBuffer buffer = pool.acquire();
+      assertTrue(buffers.contains(buffer));
+    }
+
+    assertEquals(numBuffers, pool.numCreated());
+    assertEquals(0, pool.numAvailable());
+  }
+}

+ 244 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java

@@ -0,0 +1,244 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.nio.ReadOnlyBufferException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+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;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+
+public class TestBufferData extends AbstractHadoopTestBase {
+
+  @Test
+  public void testArgChecks() throws Exception {
+    // Should not throw.
+    ByteBuffer buffer = ByteBuffer.allocate(1);
+    BufferData data = new BufferData(1, buffer);
+
+    // Verify it throws correctly.
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> new BufferData(-1, buffer));
+
+    intercept(IllegalArgumentException.class, "'buffer' must not be null",
+        () -> new BufferData(1, null));
+
+    intercept(IllegalArgumentException.class, "'actionFuture' must not be null",
+        () -> data.setPrefetch(null));
+
+    intercept(IllegalArgumentException.class, "'actionFuture' must not be null",
+        () -> data.setCaching(null));
+
+    intercept(IllegalArgumentException.class, "'states' must not be null",
+        () -> data.throwIfStateIncorrect((BufferData.State[]) null));
+
+    intercept(IllegalStateException.class,
+        "Expected buffer state to be 'READY or CACHING' but found",
+        () -> data.throwIfStateIncorrect(BufferData.State.READY,
+            BufferData.State.CACHING));
+
+  }
+
+  @Test
+  public void testValidStateUpdates() {
+    ByteBuffer buffer = ByteBuffer.allocate(1);
+    BufferData data = new BufferData(1, buffer);
+
+    assertEquals(BufferData.State.BLANK, data.getState());
+
+    CompletableFuture<Void> actionFuture = new CompletableFuture<>();
+    actionFuture.complete(null);
+    data.setPrefetch(actionFuture);
+    assertEquals(BufferData.State.PREFETCHING, data.getState());
+    assertNotNull(data.getActionFuture());
+    assertSame(actionFuture, data.getActionFuture());
+
+    CompletableFuture<Void> actionFuture2 = new CompletableFuture<>();
+    data.setCaching(actionFuture2);
+    assertEquals(BufferData.State.CACHING, data.getState());
+    assertNotNull(data.getActionFuture());
+    assertSame(actionFuture2, data.getActionFuture());
+    assertNotSame(actionFuture, actionFuture2);
+
+    List<BufferData.State> states = Arrays.asList(
+        BufferData.State.BLANK,
+        BufferData.State.PREFETCHING,
+        BufferData.State.CACHING,
+        BufferData.State.READY
+    );
+
+    BufferData data2 = new BufferData(1, buffer);
+    BufferData.State prevState = null;
+    for (BufferData.State state : states) {
+      if (prevState != null) {
+        assertEquals(prevState, data2.getState());
+        data2.updateState(state, prevState);
+        assertEquals(state, data2.getState());
+      }
+      prevState = state;
+    }
+  }
+
+  @Test
+  public void testInvalidStateUpdates() throws Exception {
+    CompletableFuture<Void> actionFuture = new CompletableFuture<>();
+    actionFuture.complete(null);
+    testInvalidStateUpdatesHelper(
+        (d) -> d.setPrefetch(actionFuture),
+        BufferData.State.BLANK,
+        BufferData.State.READY);
+
+    testInvalidStateUpdatesHelper(
+        (d) -> d.setCaching(actionFuture),
+        BufferData.State.PREFETCHING,
+        BufferData.State.READY);
+  }
+
+  @Test
+  public void testSetReady() throws Exception {
+    byte[] bytes1 = new byte[5];
+    initBytes(bytes1);
+
+    ByteBuffer buffer = ByteBuffer.allocate(10);
+    buffer.put(bytes1);
+    buffer.limit(bytes1.length);
+    BufferData data = new BufferData(1, buffer);
+    assertNotEquals(BufferData.State.READY, data.getState());
+    assertEquals(0, data.getChecksum());
+
+    data.setReady(BufferData.State.BLANK);
+    assertEquals(BufferData.State.READY, data.getState());
+    assertNotEquals(0, data.getChecksum());
+
+    // Verify that buffer cannot be modified once in READY state.
+    ExceptionAsserts.assertThrows(
+        ReadOnlyBufferException.class,
+        null,
+        () -> data.getBuffer().put(bytes1));
+
+    // Verify that buffer cannot be set to READY state more than once.
+    ExceptionAsserts.assertThrows(
+        IllegalStateException.class,
+        "Checksum cannot be changed once set",
+        () -> data.setReady(BufferData.State.BLANK));
+
+    // Verify that we detect post READY buffer modification.
+    buffer.array()[2] = (byte) 42;
+    ExceptionAsserts.assertThrows(
+        IllegalStateException.class,
+        "checksum changed after setReady()",
+        () -> data.setDone());
+  }
+
+  @Test
+  public void testChecksum() {
+    byte[] bytes1 = new byte[5];
+    byte[] bytes2 = new byte[10];
+
+    initBytes(bytes1);
+    initBytes(bytes2);
+
+    ByteBuffer buffer1 = ByteBuffer.wrap(bytes1);
+    ByteBuffer buffer2 = ByteBuffer.wrap(bytes2);
+    buffer2.limit(bytes1.length);
+
+    long checksum1 = BufferData.getChecksum(buffer1);
+    long checksum2 = BufferData.getChecksum(buffer2);
+
+    assertEquals(checksum1, checksum2);
+  }
+
+  private void initBytes(byte[] bytes) {
+    for (int i = 0; i < bytes.length; i++) {
+      bytes[i] = (byte) i;
+    }
+  }
+
+  @FunctionalInterface
+  public interface StateChanger {
+
+    void run(BufferData data) throws Exception;
+  }
+
+  private void testInvalidStateUpdatesHelper(
+      StateChanger changeState,
+      BufferData.State... validFromState) throws Exception {
+
+    ByteBuffer buffer = ByteBuffer.allocate(1);
+    BufferData data = new BufferData(1, buffer);
+    data.updateState(validFromState[0], BufferData.State.BLANK);
+    List<BufferData.State> states = this.getStatesExcept(validFromState);
+    BufferData.State prevState = validFromState[0];
+    String expectedMessage =
+        String.format("Expected buffer state to be '%s", validFromState[0]);
+    for (BufferData.State s : states) {
+      data.updateState(s, prevState);
+
+      ExceptionAsserts.assertThrows(
+          IllegalStateException.class,
+          expectedMessage,
+          () -> changeState.run(data));
+
+      assertEquals(s, data.getState());
+      prevState = s;
+    }
+  }
+
+  static final List<BufferData.State> ALL_STATES = Arrays.asList(
+      BufferData.State.UNKNOWN,
+      BufferData.State.BLANK,
+      BufferData.State.PREFETCHING,
+      BufferData.State.CACHING,
+      BufferData.State.READY
+  );
+
+  private List<BufferData.State> getStatesExcept(BufferData.State... states) {
+
+    List<BufferData.State> result = new ArrayList<>();
+    for (BufferData.State s : ALL_STATES) {
+      boolean found = false;
+      for (BufferData.State ss : states) {
+        if (s == ss) {
+          found = true;
+        }
+      }
+
+      if (!found) {
+        result.add(s);
+      }
+    }
+
+    return result;
+  }
+}

+ 158 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java

@@ -0,0 +1,158 @@
+/*
+ * 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 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.assertNotNull;
+import static org.junit.Assert.assertNull;
+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 PrefetchingStatistics statistics =
+      EmptyPrefetchingStatistics.getInstance();
+
+  @Test
+  public void testArgChecks() throws Exception {
+    // Should not throw.
+    BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, statistics);
+
+    // Verify it throws correctly.
+
+    intercept(IllegalArgumentException.class,
+        "'size' must be a positive integer",
+        () -> new BufferPool(0, 10, statistics));
+
+    intercept(IllegalArgumentException.class,
+        "'size' must be a positive integer",
+        () -> new BufferPool(-1, 10, statistics));
+
+    intercept(IllegalArgumentException.class,
+        "'bufferSize' must be a positive integer",
+        () -> new BufferPool(10, 0, statistics));
+
+    intercept(IllegalArgumentException.class,
+        "'bufferSize' must be a positive integer",
+        () -> new BufferPool(1, -10, statistics));
+
+    intercept(NullPointerException.class,
+        () -> new BufferPool(1, 10, null));
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> pool.acquire(-1));
+
+    intercept(IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> pool.tryAcquire(-1));
+
+    intercept(NullPointerException.class, "data",
+        () -> pool.release((BufferData) null));
+
+  }
+
+  @Test
+  public void testGetAndRelease() {
+    BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, statistics);
+    assertInitialState(pool, POOL_SIZE);
+
+    int count = 0;
+    for (BufferData data : pool.getAll()) {
+      count++;
+    }
+    assertEquals(0, count);
+
+    BufferData data1 = this.acquire(pool, 1);
+    BufferData data2 = this.acquire(pool, 2);
+    BufferData data3 = pool.tryAcquire(3);
+    assertNull(data3);
+
+    count = 0;
+    for (BufferData data : pool.getAll()) {
+      count++;
+    }
+    assertEquals(2, count);
+
+    assertEquals(2, pool.numCreated());
+    assertEquals(0, pool.numAvailable());
+
+    data1.updateState(BufferData.State.READY, BufferData.State.BLANK);
+    pool.release(data1);
+
+    assertEquals(2, pool.numCreated());
+    assertEquals(1, pool.numAvailable());
+
+    data2.updateState(BufferData.State.READY, BufferData.State.BLANK);
+    pool.release(data2);
+
+    assertEquals(2, pool.numCreated());
+    assertEquals(2, pool.numAvailable());
+  }
+
+  @Test
+  public void testRelease() throws Exception {
+    testReleaseHelper(BufferData.State.BLANK, true);
+    testReleaseHelper(BufferData.State.PREFETCHING, true);
+    testReleaseHelper(BufferData.State.CACHING, true);
+    testReleaseHelper(BufferData.State.READY, false);
+  }
+
+  private void testReleaseHelper(BufferData.State stateBeforeRelease,
+      boolean expectThrow)
+      throws Exception {
+
+    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) {
+
+      intercept(IllegalArgumentException.class, "Unable to release buffer",
+          () -> pool.release(data));
+
+    } else {
+      pool.release(data);
+    }
+  }
+
+  private BufferData acquire(BufferPool pool, int blockNumber) {
+    BufferData data = pool.acquire(blockNumber);
+    assertNotNull(data);
+    assertSame(data, pool.acquire(blockNumber));
+    assertEquals(blockNumber, data.getBlockNumber());
+    return data;
+  }
+
+  private void assertInitialState(BufferPool pool, int poolSize) {
+    assertEquals(poolSize, pool.numAvailable());
+    assertEquals(0, pool.numCreated());
+  }
+}

+ 98 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestExecutorServiceFuturePool.java

@@ -0,0 +1,98 @@
+/*
+ * 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.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
+import static org.junit.Assert.assertTrue;
+
+public class TestExecutorServiceFuturePool extends AbstractHadoopTestBase {
+
+  private ExecutorService executorService;
+
+  @Before
+  public void setUp() {
+    executorService = Executors.newFixedThreadPool(3);
+  }
+
+  @After
+  public void tearDown() {
+    if (executorService != null) {
+      executorService.shutdownNow();
+    }
+  }
+
+  @Test
+  public void testRunnableSucceeds() throws Exception {
+    ExecutorServiceFuturePool futurePool =
+        new ExecutorServiceFuturePool(executorService);
+    final AtomicBoolean atomicBoolean = new AtomicBoolean(false);
+    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);
+    final AtomicBoolean atomicBoolean = new AtomicBoolean(false);
+    Future<Void> future = futurePool.executeFunction(() -> {
+      atomicBoolean.set(true);
+      return null;
+    });
+    future.get(30, TimeUnit.SECONDS);
+    assertTrue("atomicBoolean set to true?", atomicBoolean.get());
+  }
+
+  @Test
+  public void testRunnableFails() throws Exception {
+    ExecutorServiceFuturePool futurePool =
+        new ExecutorServiceFuturePool(executorService);
+    Future<Void> future = futurePool.executeRunnable(() -> {
+      throw new IllegalStateException("deliberate");
+    });
+    interceptFuture(IllegalStateException.class, "deliberate", 30,
+        TimeUnit.SECONDS, future);
+  }
+
+  @Test
+  public void testSupplierFails() throws Exception {
+    ExecutorServiceFuturePool futurePool =
+        new ExecutorServiceFuturePool(executorService);
+    Future<Void> future = futurePool.executeFunction(() -> {
+      throw new IllegalStateException("deliberate");
+    });
+    interceptFuture(IllegalStateException.class, "deliberate", 30,
+        TimeUnit.SECONDS, future);
+  }
+}

+ 195 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestFilePosition.java

@@ -0,0 +1,195 @@
+/*
+ * 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.nio.ByteBuffer;
+
+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;
+
+public class TestFilePosition extends AbstractHadoopTestBase {
+
+  @Test
+  public void testArgChecks() throws Exception {
+    ByteBuffer buffer = ByteBuffer.allocate(10);
+    BufferData data = new BufferData(0, buffer);
+
+    // Should not throw.
+    new FilePosition(0, 0);
+    new FilePosition(0, 5);
+    new FilePosition(10, 5);
+    new FilePosition(5, 10);
+    new FilePosition(10, 5).setData(data, 3, 4);
+
+    // Verify it throws correctly.
+
+    intercept(IllegalArgumentException.class, "'fileSize' must not be negative",
+        () -> new FilePosition(-1, 2));
+
+    intercept(IllegalArgumentException.class,
+        "'blockSize' must be a positive integer",
+        () -> new FilePosition(1, 0));
+
+    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.
+
+    intercept(IllegalStateException.class, "'buffer' must not be null",
+        () -> pos.buffer());
+
+    intercept(IllegalStateException.class, "'buffer' must not be null",
+        () -> pos.absolute());
+
+    intercept(IllegalStateException.class, "'buffer' must not be null",
+        () -> pos.isWithinCurrentBuffer(2));
+
+    intercept(IllegalStateException.class, "'buffer' must not be null",
+        () -> pos.blockNumber());
+
+    intercept(IllegalStateException.class, "'buffer' must not be null",
+        () -> pos.isLastBlock());
+
+    intercept(IllegalStateException.class, "'buffer' must not be null",
+        () -> pos.bufferFullyRead());
+
+    // Verify that we cannot set invalid buffer parameters.
+
+    intercept(IllegalArgumentException.class, "'bufferData' must not be null",
+        () -> pos.setData(null, 4, 4));
+
+    intercept(IllegalArgumentException.class,
+        "'startOffset' must not be negative", () -> pos.setData(data, -4, 4));
+
+    intercept(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));
+
+    intercept(IllegalArgumentException.class,
+        "'readOffset' (15) must be within the range [4, 13]",
+        () -> pos.setData(data, 4, 15));
+
+    intercept(IllegalArgumentException.class,
+        "'readOffset' (3) must be within the range [4, 13]",
+        () -> pos.setData(data, 4, 3));
+
+  }
+
+  @Test
+  public void testValidity() {
+    int bufferSize = 8;
+    long fileSize = 100;
+    long bufferStartOffset = 7;
+    long readStartOffset = 9;
+
+    ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
+    BufferData data = new BufferData(0, buffer);
+    FilePosition pos = new FilePosition(fileSize, bufferSize);
+
+    assertFalse(pos.isValid());
+    pos.setData(data, bufferStartOffset, readStartOffset);
+    assertTrue(pos.isValid());
+
+    pos.invalidate();
+    assertFalse(pos.isValid());
+  }
+
+  @Test
+  public void testOffsets() {
+    int bufferSize = 8;
+    long fileSize = 100;
+    long bufferStartOffset = 7;
+    long readStartOffset = 9;
+
+    ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
+    BufferData data = new BufferData(0, buffer);
+    FilePosition pos = new FilePosition(fileSize, bufferSize);
+    pos.setData(data, bufferStartOffset, readStartOffset);
+    assertTrue(pos.isValid());
+
+    assertEquals(readStartOffset, pos.absolute());
+    assertEquals(readStartOffset - bufferStartOffset, pos.relative());
+    assertTrue(pos.isWithinCurrentBuffer(8));
+    assertFalse(pos.isWithinCurrentBuffer(6));
+    assertFalse(pos.isWithinCurrentBuffer(1));
+
+    int expectedBlockNumber = (int) (bufferStartOffset / bufferSize);
+    assertEquals(expectedBlockNumber, pos.blockNumber());
+    assertFalse(pos.isLastBlock());
+
+    pos.setData(data, fileSize - 3, fileSize - 2);
+    assertTrue(pos.isLastBlock());
+  }
+
+  @Test
+  public void testBufferStats() {
+    int bufferSize = 8;
+    long fileSize = 100;
+    long bufferStartOffset = 7;
+    long readStartOffset = 9;
+
+    ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
+    BufferData data = new BufferData(0, buffer);
+    FilePosition pos = new FilePosition(fileSize, bufferSize);
+    pos.setData(data, bufferStartOffset, readStartOffset);
+    assertTrue(pos.isValid());
+    assertEquals(bufferStartOffset, pos.bufferStartOffset());
+
+    assertEquals(0, pos.numBytesRead());
+    assertEquals(0, pos.numSingleByteReads());
+    assertEquals(0, pos.numBufferReads());
+
+    pos.incrementBytesRead(1);
+    pos.incrementBytesRead(1);
+    pos.incrementBytesRead(1);
+    pos.incrementBytesRead(5);
+    pos.incrementBytesRead(51);
+
+    assertEquals(59, pos.numBytesRead());
+    assertEquals(3, pos.numSingleByteReads());
+    assertEquals(2, pos.numBufferReads());
+
+    assertFalse(pos.bufferFullyRead());
+
+    pos.setData(data, bufferStartOffset, bufferStartOffset);
+    assertTrue(pos.isValid());
+
+    assertEquals(0, pos.numBytesRead());
+    assertEquals(0, pos.numSingleByteReads());
+    assertEquals(0, pos.numBufferReads());
+
+    for (int i = 0; i < bufferSize; i++) {
+      pos.buffer().get();
+      pos.incrementBytesRead(1);
+    }
+    assertTrue(pos.bufferFullyRead());
+  }
+}

+ 80 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestRetryer.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 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;
+
+public class TestRetryer extends AbstractHadoopTestBase {
+
+  @Test
+  public void testArgChecks() throws Exception {
+    // Should not throw.
+    new Retryer(10, 50, 500);
+
+    // Verify it throws correctly.
+
+    intercept(IllegalArgumentException.class,
+        "'perRetryDelay' must be a positive integer",
+        () -> new Retryer(-1, 50, 500));
+
+    intercept(IllegalArgumentException.class,
+        "'perRetryDelay' must be a positive integer",
+        () -> new Retryer(0, 50, 500));
+
+    intercept(IllegalArgumentException.class,
+        "'maxDelay' (5) must be greater than 'perRetryDelay' (10)",
+        () -> new Retryer(10, 5, 500));
+
+    intercept(IllegalArgumentException.class,
+        "'statusUpdateInterval' must be a positive integer",
+        () -> new Retryer(10, 50, -1));
+
+    intercept(IllegalArgumentException.class,
+        "'statusUpdateInterval' must be a positive integer",
+        () -> new Retryer(10, 50, 0));
+
+  }
+
+  @Test
+  public void testRetry() {
+    int perRetryDelay = 1;
+    int statusUpdateInterval = 3;
+    int maxDelay = 10;
+
+    Retryer retryer =
+        new Retryer(perRetryDelay, maxDelay, statusUpdateInterval);
+    for (int t = 1; t <= maxDelay; t++) {
+      assertTrue(retryer.continueRetry());
+      if (t % statusUpdateInterval == 0) {
+        assertTrue(retryer.updateStatus());
+      } else {
+        assertFalse(retryer.updateStatus());
+      }
+    }
+
+    assertFalse(retryer.continueRetry());
+  }
+}

+ 341 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestValidate.java

@@ -0,0 +1,341 @@
+/*
+ * 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.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+
+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";
+    String nullArg = null;
+
+    // Should not throw.
+    Validate.checkNotNull(nonNullArg, "nonNullArg");
+
+    // Verify it throws.
+
+    intercept(IllegalArgumentException.class, "'nullArg' must not be null",
+        () -> Validate.checkNotNull(nullArg, "nullArg"));
+
+  }
+
+  @Test
+  public void testCheckPositiveInteger() throws Exception {
+    int positiveArg = 1;
+    int zero = 0;
+    int negativeArg = -1;
+
+    // Should not throw.
+    checkPositiveInteger(positiveArg, "positiveArg");
+
+    // Verify it throws.
+
+    intercept(IllegalArgumentException.class,
+        "'negativeArg' must be a positive integer",
+        () -> checkPositiveInteger(negativeArg, "negativeArg"));
+
+    intercept(IllegalArgumentException.class,
+        "'zero' must be a positive integer",
+        () -> checkPositiveInteger(zero, "zero"));
+
+  }
+
+  @Test
+  public void testCheckNotNegative() throws Exception {
+    int positiveArg = 1;
+    int zero = 0;
+    int negativeArg = -1;
+
+    // Should not throw.
+    Validate.checkNotNegative(zero, "zeroArg");
+    Validate.checkNotNegative(positiveArg, "positiveArg");
+
+    // Verify it throws.
+
+    intercept(IllegalArgumentException.class,
+        "'negativeArg' must not be negative",
+        () -> Validate.checkNotNegative(negativeArg, "negativeArg"));
+
+  }
+
+  @Test
+  public void testCheckRequired() throws Exception {
+    // Should not throw.
+    Validate.checkRequired(true, "arg");
+
+    // Verify it throws.
+
+    intercept(IllegalArgumentException.class, "'arg' is required",
+        () -> Validate.checkRequired(false, "arg"));
+
+  }
+
+  @Test
+  public void testCheckValid() throws Exception {
+    // Should not throw.
+    Validate.checkValid(true, "arg");
+
+    // Verify it throws.
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'arg' is invalid",
+        () -> Validate.checkValid(false, "arg"));
+  }
+
+  @Test
+  public void testCheckValidWithValues() throws Exception {
+    String validValues = "foo, bar";
+
+    // Should not throw.
+    Validate.checkValid(true, "arg", validValues);
+
+    // Verify it throws.
+
+    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(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.
+
+    intercept(IllegalArgumentException.class, "'string' must not be empty",
+        () -> Validate.checkNotNullAndNotEmpty("", "string"));
+
+    intercept(IllegalArgumentException.class, "'array' must not be null", () ->
+        Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_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(NULL_BYTE_ARRAY, "array"));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'array' must have at least one element",
+        () -> Validate.checkNotNullAndNotEmpty(EMPTY_BYTE_ARRAY, "array"));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'array' must not be null",
+        () -> Validate.checkNotNullAndNotEmpty(NULL_SHORT_ARRAY, "array"));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'array' must have at least one element",
+        () -> Validate.checkNotNullAndNotEmpty(EMPTY_SHORT_ARRAY, "array"));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'array' must not be null",
+        () -> Validate.checkNotNullAndNotEmpty(NULL_INT_ARRAY, "array"));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'array' must have at least one element",
+        () -> Validate.checkNotNullAndNotEmpty(EMPTY_INT_ARRAY, "array"));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'array' must not be null",
+        () -> Validate.checkNotNullAndNotEmpty(NULL_LONG_ARRAY, "array"));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'array' must have at least one element",
+        () -> Validate.checkNotNullAndNotEmpty(EMPTY_LONG_ARRAY, "array"));
+  }
+
+  @Test
+  public void testCheckListNotNullAndNotEmpty() throws Exception {
+    // Should not throw.
+    Validate.checkNotNullAndNotEmpty(VALID_LIST, "list");
+
+    // Verify it throws.
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'list' must not be null",
+        () -> Validate.checkNotNullAndNotEmpty(NULL_LIST, "list"));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'list' must have at least one element",
+        () -> Validate.checkNotNullAndNotEmpty(EMPTY_LIST, "list"));
+  }
+
+  @Test
+  public void testCheckNotNullAndNumberOfElements() throws Exception {
+    // Should not throw.
+    Validate.checkNotNullAndNumberOfElements(Arrays.asList(1, 2, 3), 3, "arg");
+
+    // Verify it throws.
+
+    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")
+    );
+  }
+
+  @Test
+  public void testCheckValuesEqual() throws Exception {
+    // Should not throw.
+    Validate.checkValuesEqual(1, "arg1", 1, "arg2");
+
+    // Verify it throws.
+
+    intercept(IllegalArgumentException.class,
+        "'arg1' (1) must equal 'arg2' (2)",
+        () -> Validate.checkValuesEqual(1, "arg1", 2, "arg2"));
+
+  }
+
+  @Test
+  public void testCheckIntegerMultiple() throws Exception {
+    // Should not throw.
+    Validate.checkIntegerMultiple(10, "arg1", 5, "arg2");
+
+    // Verify it throws.
+
+    intercept(IllegalArgumentException.class,
+        "'arg1' (10) must be an integer multiple of 'arg2' (3)",
+        () -> Validate.checkIntegerMultiple(10, "arg1", 3, "arg2"));
+
+  }
+
+  @Test
+  public void testCheckGreater() throws Exception {
+    // Should not throw.
+    Validate.checkGreater(10, "arg1", 5, "arg2");
+
+    // Verify it throws.
+
+    intercept(IllegalArgumentException.class,
+        "'arg1' (5) must be greater than 'arg2' (10)",
+        () -> Validate.checkGreater(5, "arg1", 10, "arg2"));
+
+  }
+
+  @Test
+  public void testCheckGreaterOrEqual() throws Exception {
+    // Should not throw.
+    Validate.checkGreaterOrEqual(10, "arg1", 5, "arg2");
+
+    // Verify it throws.
+
+    intercept(IllegalArgumentException.class,
+        "'arg1' (5) must be greater than or equal to 'arg2' (10)",
+        () -> Validate.checkGreaterOrEqual(5, "arg1", 10, "arg2"));
+
+  }
+
+  @Test
+  public void testCheckWithinRange() throws Exception {
+    // Should not throw.
+    Validate.checkWithinRange(10, "arg", 5, 15);
+    Validate.checkWithinRange(10.0, "arg", 5.0, 15.0);
+
+    // Verify it throws.
+
+    intercept(IllegalArgumentException.class,
+        "'arg' (5) must be within the range [10, 20]",
+        () -> Validate.checkWithinRange(5, "arg", 10, 20));
+
+    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 notFound = Paths.get("<not-found>");
+
+    // Should not throw.
+    Validate.checkPathExists(tempFile, "tempFile");
+    Validate.checkPathExists(tempDir, "tempDir");
+
+    // Verify it throws.
+
+    intercept(IllegalArgumentException.class, "'nullArg' must not be null",
+        () -> Validate.checkPathExists(null, "nullArg"));
+
+    intercept(IllegalArgumentException.class,
+        "Path notFound (<not-found>) does not exist",
+        () -> Validate.checkPathExists(notFound, "notFound"));
+
+    intercept(IllegalArgumentException.class, "must point to a directory",
+        () -> Validate.checkPathExistsAsDir(tempFile, "tempFile"));
+
+    intercept(IllegalArgumentException.class, "must point to a file",
+        () -> Validate.checkPathExistsAsFile(tempDir, "tempDir"));
+
+  }
+}

+ 25 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java

@@ -1208,6 +1208,31 @@ public final class Constants {
    */
   public static final int DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE = 1253376; //1M
 
+  /**
+   * Controls whether the prefetching input stream is enabled.
+   */
+  public static final String PREFETCH_ENABLED_KEY = "fs.s3a.prefetch.enabled";
+
+  /**
+   * Default option as to whether the prefetching input stream is enabled.
+   */
+  public static final boolean  PREFETCH_ENABLED_DEFAULT = false;
+
+  // If the default values are used, each file opened for reading will consume
+  // 64 MB of heap space (8 blocks x 8 MB each).
+
+  /**
+   * The size of a single prefetched block in number of bytes.
+   */
+  public static final String PREFETCH_BLOCK_SIZE_KEY = "fs.s3a.prefetch.block.size";
+  public static final int PREFETCH_BLOCK_DEFAULT_SIZE = 8 * 1024 * 1024;
+
+  /**
+   * Maximum number of blocks prefetched at any given time.
+   */
+  public static final String PREFETCH_BLOCK_COUNT_KEY = "fs.s3a.prefetch.block.count";
+  public static final int PREFETCH_BLOCK_DEFAULT_COUNT = 8;
+
   /**
    * Prefix of auth classes in AWS SDK V1.
    */

+ 59 - 14
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -85,6 +85,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.impl.prefetch.ExecutorServiceFuturePool;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -131,6 +132,7 @@ 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.impl.V2Migration;
+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;
@@ -295,6 +297,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   private TransferManager transfers;
   private ExecutorService boundedThreadPool;
   private ThreadPoolExecutor unboundedThreadPool;
+
+  // S3 reads are prefetched asynchronously using this future pool.
+  private ExecutorServiceFuturePool futurePool;
+
+  // If true, the prefetching input stream is used for reads.
+  private boolean prefetchEnabled;
+
+  // Size in bytes of a single prefetch block.
+  private int prefetchBlockSize;
+
+  // Size of prefetch queue (in number of blocks).
+  private int prefetchBlockCount;
+
   private int executorCapacity;
   private long multiPartThreshold;
   public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
@@ -519,6 +534,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       longBytesOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1);
       enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
 
+      this.prefetchEnabled = conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT);
+      this.prefetchBlockSize = intOption(
+          conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, PREFETCH_BLOCK_DEFAULT_SIZE);
+      this.prefetchBlockCount =
+          intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1);
+
       initThreadPools(conf);
 
       int listVersion = conf.getInt(LIST_VERSION, DEFAULT_LIST_VERSION);
@@ -627,11 +648,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       // amazon client exception: stop all services then throw the translation
       cleanupWithLogger(LOG, span);
       stopAllServices();
+      if (this.futurePool != null) {
+        this.futurePool = null;
+      }
       throw translateException("initializing ", new Path(name), e);
     } catch (IOException | RuntimeException e) {
       // other exceptions: stop the services.
       cleanupWithLogger(LOG, span);
       stopAllServices();
+      if (this.futurePool != null) {
+        this.futurePool = null;
+      }
       throw e;
     }
   }
@@ -752,9 +779,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS, 1);
     long keepAliveTime = longOption(conf, KEEPALIVE_TIME,
         DEFAULT_KEEPALIVE_TIME, 0);
+    int numPrefetchThreads = this.prefetchEnabled ? this.prefetchBlockCount : 0;
+
     boundedThreadPool = BlockingThreadPoolExecutorService.newInstance(
         maxThreads,
-        maxThreads + totalTasks,
+        maxThreads + totalTasks + numPrefetchThreads,
         keepAliveTime, TimeUnit.SECONDS,
         name + "-bounded");
     unboundedThreadPool = new ThreadPoolExecutor(
@@ -766,6 +795,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     unboundedThreadPool.allowCoreThreadTimeOut(true);
     executorCapacity = intOption(conf,
         EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1);
+    if (this.prefetchEnabled) {
+      this.futurePool = new ExecutorServiceFuturePool(boundedThreadPool);
+    }
   }
 
   /**
@@ -1526,16 +1558,26 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         auditSpan);
     fileInformation.applyOptions(readContext);
     LOG.debug("Opening '{}'", readContext);
-    return new FSDataInputStream(
-            new S3AInputStream(
-                  readContext.build(),
-                  createObjectAttributes(path, fileStatus),
-                  createInputStreamCallbacks(auditSpan),
-                  inputStreamStats,
-                  new SemaphoredDelegatingExecutor(
-                          boundedThreadPool,
-                          vectoredActiveRangeReads,
-                          true)));
+
+    if (this.prefetchEnabled) {
+      return new FSDataInputStream(
+          new S3APrefetchingInputStream(
+              readContext.build(),
+              createObjectAttributes(path, fileStatus),
+              createInputStreamCallbacks(auditSpan),
+              inputStreamStats));
+    } else {
+      return new FSDataInputStream(
+          new S3AInputStream(
+              readContext.build(),
+              createObjectAttributes(path, fileStatus),
+              createInputStreamCallbacks(auditSpan),
+              inputStreamStats,
+              new SemaphoredDelegatingExecutor(
+                  boundedThreadPool,
+                  vectoredActiveRangeReads,
+                  true)));
+    }
   }
 
   /**
@@ -1643,7 +1685,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         statisticsContext,
         fileStatus,
         vectoredIOContext,
-        IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator())
+        IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator(),
+        futurePool,
+        prefetchBlockSize,
+        prefetchBlockCount)
         .withAuditSpan(auditSpan);
     openFileHelper.applyDefaultOptions(roc);
     return roc.build();
@@ -3760,12 +3805,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   protected CopyFromLocalOperation.CopyFromLocalOperationCallbacks
-  createCopyFromLocalCallbacks() throws IOException {
+      createCopyFromLocalCallbacks() throws IOException {
     LocalFileSystem local = getLocal(getConf());
     return new CopyFromLocalCallbacksImpl(local);
   }
 
-  protected class CopyFromLocalCallbacksImpl implements
+  protected final class CopyFromLocalCallbacksImpl implements
       CopyFromLocalOperation.CopyFromLocalOperationCallbacks {
     private final LocalFileSystem local;
 

+ 59 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java

@@ -887,11 +887,19 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
               StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
               StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED,
               StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES)
-          .withGauges(STREAM_READ_GAUGE_INPUT_POLICY)
+          .withGauges(STREAM_READ_GAUGE_INPUT_POLICY,
+              STREAM_READ_BLOCKS_IN_FILE_CACHE.getSymbol(),
+              STREAM_READ_ACTIVE_PREFETCH_OPERATIONS.getSymbol(),
+              STREAM_READ_ACTIVE_MEMORY_IN_USE.getSymbol()
+              )
           .withDurationTracking(ACTION_HTTP_GET_REQUEST,
+              ACTION_EXECUTOR_ACQUIRED,
               StoreStatisticNames.ACTION_FILE_OPENED,
               StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED,
-              StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED)
+              StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED,
+              StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS,
+              StreamStatisticNames.STREAM_READ_REMOTE_BLOCK_READ,
+              StreamStatisticNames.STREAM_READ_BLOCK_ACQUIRE_AND_READ)
           .build();
       setIOStatistics(st);
       aborted = st.getCounterReference(
@@ -960,6 +968,18 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
       return incCounter(name, value);
     }
 
+    /**
+     * Increment the Statistic gauge and the local IOStatistics
+     * equivalent.
+     * @param statistic statistic
+     * @param v value.
+     * @return local IOStatistic value
+     */
+    private long incAllGauges(Statistic statistic, long v) {
+      incrementGauge(statistic, v);
+      return incGauge(statistic.getSymbol(), v);
+    }
+
     /**
      * {@inheritDoc}.
      * Increments the number of seek operations,
@@ -1088,6 +1108,12 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
       bytesDiscardedInVectoredIO.addAndGet(discarded);
     }
 
+    @Override
+    public void executorAcquired(Duration timeInQueue) {
+      // update the duration fields in the IOStatistics.
+      localIOStatistics().addTimedOperation(ACTION_EXECUTOR_ACQUIRED, timeInQueue);
+    }
+
     /**
      * {@code close()} merges the stream statistics into the filesystem's
      * instrumentation instance.
@@ -1352,6 +1378,37 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
           ? StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED
           : StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED);
     }
+
+    @Override
+    public DurationTracker prefetchOperationStarted() {
+      incAllGauges(STREAM_READ_ACTIVE_PREFETCH_OPERATIONS, 1);
+      return trackDuration(StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS);
+    }
+
+    @Override
+    public void blockAddedToFileCache() {
+      incAllGauges(STREAM_READ_BLOCKS_IN_FILE_CACHE, 1);
+    }
+
+    @Override
+    public void blockRemovedFromFileCache() {
+      incAllGauges(STREAM_READ_BLOCKS_IN_FILE_CACHE, -1);
+    }
+
+    @Override
+    public void prefetchOperationCompleted() {
+      incAllGauges(STREAM_READ_ACTIVE_PREFETCH_OPERATIONS, -1);
+    }
+
+    @Override
+    public void memoryAllocated(int size) {
+      incAllGauges(STREAM_READ_ACTIVE_MEMORY_IN_USE, size);
+    }
+
+    @Override
+    public void memoryFreed(int size) {
+      incAllGauges(STREAM_READ_ACTIVE_MEMORY_IN_USE, -size);
+    }
   }
 
   /**

+ 52 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java

@@ -21,6 +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.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;
@@ -74,6 +75,15 @@ public class S3AReadOpContext extends S3AOpContext {
   /** Thread-level IOStatistics aggregator. **/
   private final IOStatisticsAggregator ioStatisticsAggregator;
 
+  // S3 reads are prefetched asynchronously using this future pool.
+  private ExecutorServiceFuturePool futurePool;
+
+  // Size in bytes of a single prefetch block.
+  private final int prefetchBlockSize;
+
+  // Size of prefetch queue (in number of blocks).
+  private final int prefetchBlockCount;
+
   /**
    * Instantiate.
    * @param path path of read
@@ -83,6 +93,9 @@ public class S3AReadOpContext extends S3AOpContext {
    * @param dstFileStatus target file status
    * @param vectoredIOContext context for vectored read operation.
    * @param ioStatisticsAggregator IOStatistics aggregator for each thread.
+   * @param futurePool the ExecutorServiceFuturePool instance used by async prefetches.
+   * @param prefetchBlockSize the size (in number of bytes) of each prefetched block.
+   * @param prefetchBlockCount maximum number of prefetched blocks.
    */
   public S3AReadOpContext(
       final Path path,
@@ -91,12 +104,23 @@ public class S3AReadOpContext extends S3AOpContext {
       S3AStatisticsContext instrumentation,
       FileStatus dstFileStatus,
       VectoredIOContext vectoredIOContext,
-      IOStatisticsAggregator ioStatisticsAggregator) {
+      IOStatisticsAggregator ioStatisticsAggregator,
+      ExecutorServiceFuturePool futurePool,
+      int prefetchBlockSize,
+      int prefetchBlockCount) {
+
     super(invoker, stats, instrumentation,
         dstFileStatus);
     this.path = requireNonNull(path);
     this.vectoredIOContext = requireNonNull(vectoredIOContext, "vectoredIOContext");
     this.ioStatisticsAggregator = ioStatisticsAggregator;
+    this.futurePool = futurePool;
+    Preconditions.checkArgument(
+        prefetchBlockSize > 0, "invalid prefetchBlockSize %d", prefetchBlockSize);
+    this.prefetchBlockSize = prefetchBlockSize;
+    Preconditions.checkArgument(
+        prefetchBlockCount > 0, "invalid prefetchBlockCount %d", prefetchBlockCount);
+    this.prefetchBlockCount = prefetchBlockCount;
   }
 
   /**
@@ -232,6 +256,33 @@ public class S3AReadOpContext extends S3AOpContext {
     return ioStatisticsAggregator;
   }
 
+  /**
+   * Gets the {@code ExecutorServiceFuturePool} used for asynchronous prefetches.
+   *
+   * @return the {@code ExecutorServiceFuturePool} used for asynchronous prefetches.
+   */
+  public ExecutorServiceFuturePool getFuturePool() {
+    return this.futurePool;
+  }
+
+  /**
+   * Gets the size in bytes of a single prefetch block.
+   *
+   * @return the size in bytes of a single prefetch block.
+   */
+  public int getPrefetchBlockSize() {
+    return this.prefetchBlockSize;
+  }
+
+  /**
+   * Gets the size of prefetch queue (in number of blocks).
+   *
+   * @return the size of prefetch queue (in number of blocks).
+   */
+  public int getPrefetchBlockCount() {
+    return this.prefetchBlockCount;
+  }
+
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder(

+ 12 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java

@@ -382,6 +382,18 @@ public enum Statistic {
       StreamStatisticNames.STREAM_READ_UNBUFFERED,
       "Total count of input stream unbuffering operations",
       TYPE_COUNTER),
+  STREAM_READ_BLOCKS_IN_FILE_CACHE(
+      StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE,
+      "Gauge of blocks in disk cache",
+      TYPE_GAUGE),
+  STREAM_READ_ACTIVE_PREFETCH_OPERATIONS(
+      StreamStatisticNames.STREAM_READ_ACTIVE_PREFETCH_OPERATIONS,
+      "Gauge of active prefetches",
+      TYPE_GAUGE),
+  STREAM_READ_ACTIVE_MEMORY_IN_USE(
+      StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE,
+      "Gauge of active memory in use",
+      TYPE_GAUGE),
 
   /* Stream Write statistics */
 

+ 78 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ABlockManager.java

@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+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 S3ACachingBlockManager}.
+ */
+public class S3ABlockManager extends BlockManager {
+
+  /**
+   * Reader that reads from S3 file.
+   */
+  private final S3ARemoteObjectReader reader;
+
+  /**
+   * 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.
+   *
+   * @throws IllegalArgumentException if reader is null.
+   * @throws IllegalArgumentException if blockData is null.
+   */
+  public S3ABlockManager(S3ARemoteObjectReader reader, BlockData blockData) {
+    super(blockData);
+
+    Validate.checkNotNull(reader, "reader");
+
+    this.reader = reader;
+  }
+
+  /**
+   * Reads into the given {@code buffer} {@code size} bytes from the underlying file
+   * starting at {@code startOffset}.
+   *
+   * @param buffer the buffer to read data in to.
+   * @param startOffset the offset at which reading starts.
+   * @param size the number bytes to read.
+   * @return number of bytes read.
+   */
+  @Override
+  public int read(ByteBuffer buffer, long startOffset, int size)
+      throws IOException {
+    return reader.read(buffer, startOffset, size);
+  }
+
+  @Override
+  public void close() {
+    reader.close();
+  }
+}

+ 96 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingBlockManager.java

@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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 S3ACachingBlockManager extends CachingBlockManager {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      S3ACachingBlockManager.class);
+
+  /**
+   * Reader that reads from S3 file.
+   */
+  private final S3ARemoteObjectReader reader;
+
+  /**
+   * Constructs an instance of a {@code S3ACachingBlockManager}.
+   *
+   * @param futurePool asynchronous tasks are performed in this pool.
+   * @param reader reader that reads from S3 file.
+   * @param blockData information about each block of the S3 file.
+   * @param bufferPoolSize size of the in-memory cache in terms of number of blocks.
+   * @param streamStatistics statistics for this stream.
+   *
+   * @throws IllegalArgumentException if reader is null.
+   */
+  public S3ACachingBlockManager(
+      ExecutorServiceFuturePool futurePool,
+      S3ARemoteObjectReader reader,
+      BlockData blockData,
+      int bufferPoolSize,
+      S3AInputStreamStatistics streamStatistics) {
+    super(futurePool, blockData, bufferPoolSize, streamStatistics);
+
+    Validate.checkNotNull(reader, "reader");
+
+    this.reader = reader;
+  }
+
+  protected S3ARemoteObjectReader getReader() {
+    return this.reader;
+  }
+
+  /**
+   * Reads into the given {@code buffer} {@code size} bytes from the underlying file
+   * starting at {@code startOffset}.
+   *
+   * @param buffer the buffer to read data in to.
+   * @param startOffset the offset at which reading starts.
+   * @param size the number bytes to read.
+   * @return number of bytes read.
+   */
+  @Override
+  public int read(ByteBuffer buffer, long startOffset, int size)
+      throws IOException {
+    return this.reader.read(buffer, startOffset, size);
+  }
+
+  @Override
+  public synchronized void close() {
+    this.reader.close();
+
+    super.close();
+  }
+}

+ 214 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java

@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCK_ACQUIRE_AND_READ;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
+
+/**
+ * Provides an {@code InputStream} that allows reading from an S3 file.
+ * Prefetched blocks are cached to local disk if a seek away from the
+ * current block is issued.
+ */
+public class S3ACachingInputStream extends S3ARemoteInputStream {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      S3ACachingInputStream.class);
+
+  /**
+   * Number of blocks queued for prefching.
+   */
+  private final int numBlocksToPrefetch;
+
+  private final BlockManager blockManager;
+
+  /**
+   * Initializes a new instance of the {@code S3ACachingInputStream} class.
+   *
+   * @param context read-specific operation context.
+   * @param s3Attributes attributes of the S3 object being read.
+   * @param client callbacks used for interacting with the underlying S3 client.
+   * @param streamStatistics statistics for this stream.
+   *
+   * @throws IllegalArgumentException if context is null.
+   * @throws IllegalArgumentException if s3Attributes is null.
+   * @throws IllegalArgumentException if client is null.
+   */
+  public S3ACachingInputStream(
+      S3AReadOpContext context,
+      S3ObjectAttributes s3Attributes,
+      S3AInputStream.InputStreamCallbacks client,
+      S3AInputStreamStatistics streamStatistics) {
+    super(context, s3Attributes, client, streamStatistics);
+
+    this.numBlocksToPrefetch = this.getContext().getPrefetchBlockCount();
+    int bufferPoolSize = this.numBlocksToPrefetch + 1;
+    this.blockManager = this.createBlockManager(
+        this.getContext().getFuturePool(),
+        this.getReader(),
+        this.getBlockData(),
+        bufferPoolSize);
+    int fileSize = (int) s3Attributes.getLen();
+    LOG.debug("Created caching input stream for {} (size = {})", this.getName(),
+        fileSize);
+  }
+
+  /**
+   * Moves the current read position so that the next read will occur at {@code pos}.
+   *
+   * @param pos the next read will take place at this position.
+   *
+   * @throws IllegalArgumentException if pos is outside of the range [0, file size].
+   */
+  @Override
+  public void seek(long pos) throws IOException {
+    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 (!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 (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;
+        //    therefore we release the buffer without caching.
+        // -- 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 (!getFilePosition().bufferFullyRead()) {
+          blockManager.requestCaching(getFilePosition().data());
+        } else {
+          blockManager.release(getFilePosition().data());
+        }
+        getFilePosition().invalidate();
+        blockManager.cancelPrefetches();
+      }
+      setSeekTargetPos(pos);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    // Close the BlockManager first, cancelling active prefetches,
+    // deleting cached files and freeing memory used by buffer pool.
+    blockManager.close();
+    super.close();
+    LOG.info("closed: {}", getName());
+  }
+
+  @Override
+  protected boolean ensureCurrentBuffer() throws IOException {
+    if (isClosed()) {
+      return false;
+    }
+
+    if (getFilePosition().isValid() && getFilePosition()
+        .buffer()
+        .hasRemaining()) {
+      return true;
+    }
+
+    long readPos;
+    int prefetchCount;
+
+    if (getFilePosition().isValid()) {
+      // A sequential read results in a prefetch.
+      readPos = getFilePosition().absolute();
+      prefetchCount = numBlocksToPrefetch;
+    } else {
+      // A seek invalidates the current position.
+      // We prefetch only 1 block immediately after a seek operation.
+      readPos = getSeekTargetPos();
+      prefetchCount = 1;
+    }
+
+    if (!getBlockData().isValidOffset(readPos)) {
+      return false;
+    }
+
+    if (getFilePosition().isValid()) {
+      if (getFilePosition().bufferFullyRead()) {
+        blockManager.release(getFilePosition().data());
+      } else {
+        blockManager.requestCaching(getFilePosition().data());
+      }
+    }
+
+    int toBlockNumber = getBlockData().getBlockNumber(readPos);
+    long startOffset = getBlockData().getStartOffset(toBlockNumber);
+
+    for (int i = 1; i <= prefetchCount; i++) {
+      int b = toBlockNumber + i;
+      if (b < getBlockData().getNumBlocks()) {
+        blockManager.requestPrefetch(b);
+      }
+    }
+
+    BufferData data = invokeTrackingDuration(
+        getS3AStreamStatistics()
+            .trackDuration(STREAM_READ_BLOCK_ACQUIRE_AND_READ),
+        () -> blockManager.get(toBlockNumber));
+
+    getFilePosition().setData(data, startOffset, readPos);
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    if (isClosed()) {
+      return "closed";
+    }
+
+    StringBuilder sb = new StringBuilder();
+    sb.append(String.format("fpos = (%s)%n", getFilePosition()));
+    sb.append(blockManager.toString());
+    return sb.toString();
+  }
+
+  protected BlockManager createBlockManager(
+      ExecutorServiceFuturePool futurePool,
+      S3ARemoteObjectReader reader,
+      BlockData blockData,
+      int bufferPoolSize) {
+    return new S3ACachingBlockManager(futurePool, reader, blockData,
+        bufferPoolSize,
+        getS3AStreamStatistics());
+  }
+}

+ 102 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java

@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+
+/**
+ * Provides an {@code InputStream} that allows reading from an S3 file.
+ * The entire file is read into memory before reads can begin.
+ *
+ * Use of this class is recommended only for small files that can fit
+ * entirely in memory.
+ */
+public class S3AInMemoryInputStream extends S3ARemoteInputStream {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      S3AInMemoryInputStream.class);
+
+  private ByteBuffer buffer;
+
+  /**
+   * Initializes a new instance of the {@code S3AInMemoryInputStream} class.
+   *
+   * @param context read-specific operation context.
+   * @param s3Attributes attributes of the S3 object being read.
+   * @param client callbacks used for interacting with the underlying S3 client.
+   * @param streamStatistics statistics for this stream.
+   *
+   * @throws IllegalArgumentException if context is null.
+   * @throws IllegalArgumentException if s3Attributes is null.
+   * @throws IllegalArgumentException if client is null.
+   */
+  public S3AInMemoryInputStream(
+      S3AReadOpContext context,
+      S3ObjectAttributes s3Attributes,
+      S3AInputStream.InputStreamCallbacks client,
+      S3AInputStreamStatistics streamStatistics) {
+    super(context, s3Attributes, client, streamStatistics);
+    int fileSize = (int) s3Attributes.getLen();
+    this.buffer = ByteBuffer.allocate(fileSize);
+    LOG.debug("Created in-memory input stream for {} (size = {})",
+        getName(), fileSize);
+  }
+
+  /**
+   * Ensures that a non-empty valid buffer is available for immediate reading.
+   * It returns true when at least one such buffer is available for reading.
+   * It returns false on reaching the end of the stream.
+   *
+   * @return true if at least one such buffer is available for reading, false otherwise.
+   */
+  @Override
+  protected boolean ensureCurrentBuffer() throws IOException {
+    if (isClosed()) {
+      return false;
+    }
+
+    if (getBlockData().getFileSize() == 0) {
+      return false;
+    }
+
+    if (!getFilePosition().isValid()) {
+      buffer.clear();
+      int numBytesRead =
+          getReader().read(buffer, 0, buffer.capacity());
+      if (numBytesRead <= 0) {
+        return false;
+      }
+      BufferData data = new BufferData(0, buffer);
+      getFilePosition().setData(data, 0, getSeekTargetPos());
+    }
+
+    return getFilePosition().buffer().hasRemaining();
+  }
+}

+ 260 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java

@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+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.impl.prefetch.Validate;
+import org.apache.hadoop.fs.s3a.S3AInputStream;
+import org.apache.hadoop.fs.s3a.S3AReadOpContext;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+/**
+ * Enhanced {@code InputStream} for reading from S3.
+ *
+ * This implementation provides improved read throughput by asynchronously prefetching
+ * blocks of configurable size from the underlying S3 file.
+ */
+public class S3APrefetchingInputStream
+    extends FSInputStream
+    implements CanSetReadahead, StreamCapabilities, IOStatisticsSource {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      S3APrefetchingInputStream.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.
+   * @param client callbacks used for interacting with the underlying S3 client.
+   * @param streamStatistics statistics for this stream.
+   *
+   * @throws IllegalArgumentException if context is null.
+   * @throws IllegalArgumentException if s3Attributes is null.
+   * @throws IllegalArgumentException if client is null.
+   */
+  public S3APrefetchingInputStream(
+      S3AReadOpContext context,
+      S3ObjectAttributes s3Attributes,
+      S3AInputStream.InputStreamCallbacks client,
+      S3AInputStreamStatistics streamStatistics) {
+
+    Validate.checkNotNull(context, "context");
+    Validate.checkNotNull(s3Attributes, "s3Attributes");
+    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()) {
+      LOG.debug("Creating in memory input stream for {}", context.getPath());
+      this.inputStream = new S3AInMemoryInputStream(
+          context,
+          s3Attributes,
+          client,
+          streamStatistics);
+    } else {
+      LOG.debug("Creating in caching input stream for {}", context.getPath());
+      this.inputStream = new S3ACachingInputStream(
+          context,
+          s3Attributes,
+          client,
+          streamStatistics);
+    }
+  }
+
+  /**
+   * Returns the number of bytes available for reading without blocking.
+   *
+   * @return the number of bytes available for reading without blocking.
+   * @throws IOException if there is an IO error during this operation.
+   */
+  @Override
+  public synchronized int available() throws IOException {
+    throwIfClosed();
+    return inputStream.available();
+  }
+
+  /**
+   * Gets the current position.
+   *
+   * @return the current position.
+   * @throws IOException if there is an IO error during this operation.
+   */
+  @Override
+  public synchronized long getPos() throws IOException {
+    return isClosed() ? 0 : inputStream.getPos();
+  }
+
+  /**
+   * Reads and returns one byte from this stream.
+   *
+   * @return the next byte from this stream.
+   * @throws IOException if there is an IO error during this operation.
+   */
+  @Override
+  public synchronized int read() throws IOException {
+    throwIfClosed();
+    return inputStream.read();
+  }
+
+  /**
+   * Reads up to {@code len} bytes from this stream and copies them into
+   * the given {@code buffer} starting at the given {@code offset}.
+   * Returns the number of bytes actually copied in to the given buffer.
+   *
+   * @param buffer the buffer to copy data into.
+   * @param offset data is copied starting at this offset.
+   * @param len max number of bytes to copy.
+   * @return the number of bytes actually copied in to the given buffer.
+   * @throws IOException if there is an IO error during this operation.
+   */
+  @Override
+  public synchronized int read(byte[] buffer, int offset, int len)
+      throws IOException {
+    throwIfClosed();
+    return inputStream.read(buffer, offset, len);
+  }
+
+  /**
+   * Closes this stream and releases all acquired resources.
+   *
+   * @throws IOException if there is an IO error during this operation.
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    if (inputStream != null) {
+      inputStream.close();
+      inputStream = null;
+      super.close();
+    }
+  }
+
+  /**
+   * Updates internal data such that the next read will take place at the given {@code pos}.
+   *
+   * @param pos new read position.
+   * @throws IOException if there is an IO error during this operation.
+   */
+  @Override
+  public synchronized void seek(long pos) throws IOException {
+    throwIfClosed();
+    inputStream.seek(pos);
+  }
+
+  /**
+   * Sets the number of bytes to read ahead each time.
+   *
+   * @param readahead the number of bytes to read ahead each time..
+   */
+  @Override
+  public synchronized void setReadahead(Long readahead) {
+    if (!isClosed()) {
+      inputStream.setReadahead(readahead);
+    }
+  }
+
+  /**
+   * Indicates whether the given {@code capability} is supported by this stream.
+   *
+   * @param capability the capability to check.
+   * @return true if the given {@code capability} is supported by this stream, false otherwise.
+   */
+  @Override
+  public boolean hasCapability(String capability) {
+    if (!isClosed()) {
+      return inputStream.hasCapability(capability);
+    }
+
+    return false;
+  }
+
+  /**
+   * Access the input stream statistics.
+   * This is for internal testing and may be removed without warning.
+   * @return the statistics for this input stream
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public S3AInputStreamStatistics getS3AStreamStatistics() {
+    if (isClosed()) {
+      return null;
+    }
+    return inputStream.getS3AStreamStatistics();
+  }
+
+  /**
+   * Gets the internal IO statistics.
+   *
+   * @return the internal IO statistics.
+   */
+  @Override
+  public IOStatistics getIOStatistics() {
+    if (isClosed()) {
+      return null;
+    }
+    return inputStream.getIOStatistics();
+  }
+
+  protected boolean isClosed() {
+    return inputStream == null;
+  }
+
+  protected void throwIfClosed() throws IOException {
+    if (isClosed()) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+  }
+
+  // Unsupported functions.
+
+  @Override
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    throwIfClosed();
+    return false;
+  }
+
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+}

+ 475 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java

@@ -0,0 +1,475 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+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.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;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Provides an {@link InputStream} that allows reading from an S3 file.
+ */
+public abstract class S3ARemoteInputStream
+    extends InputStream
+    implements CanSetReadahead, StreamCapabilities, IOStatisticsSource {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      S3ARemoteInputStream.class);
+
+  /**
+   * The S3 file read by this instance.
+   */
+  private S3ARemoteObject remoteObject;
+
+  /**
+   * Reading of S3 file takes place through this reader.
+   */
+  private S3ARemoteObjectReader reader;
+
+  /**
+   * Name of this stream. Used only for logging.
+   */
+  private final String name;
+
+  /**
+   * Indicates whether the stream has been closed.
+   */
+  private volatile boolean closed;
+
+  /**
+   * Current position within the file.
+   */
+  private FilePosition fpos;
+
+  /** The target of the most recent seek operation. */
+  private long seekTargetPos;
+
+  /** Information about each block of the mapped S3 file. */
+  private BlockData blockData;
+
+  /** Read-specific operation context. */
+  private S3AReadOpContext context;
+
+  /** Attributes of the S3 object being read. */
+  private S3ObjectAttributes s3Attributes;
+
+  /** Callbacks used for interacting with the underlying S3 client. */
+  private S3AInputStream.InputStreamCallbacks client;
+
+  /** 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 S3ARemoteInputStream} class.
+   *
+   * @param context read-specific operation context.
+   * @param s3Attributes attributes of the S3 object being read.
+   * @param client callbacks used for interacting with the underlying S3 client.
+   * @param streamStatistics statistics for this stream.
+   *
+   * @throws IllegalArgumentException if context is null.
+   * @throws IllegalArgumentException if s3Attributes is null.
+   * @throws IllegalArgumentException if client is null.
+   */
+  public S3ARemoteInputStream(
+      S3AReadOpContext context,
+      S3ObjectAttributes s3Attributes,
+      S3AInputStream.InputStreamCallbacks client,
+      S3AInputStreamStatistics streamStatistics) {
+
+    this.context = requireNonNull(context);
+    this.s3Attributes = requireNonNull(s3Attributes);
+    this.client = requireNonNull(client);
+    this.streamStatistics = requireNonNull(streamStatistics);
+    this.ioStatistics = streamStatistics.getIOStatistics();
+    this.name = S3ARemoteObject.getPath(s3Attributes);
+    this.changeTracker = new ChangeTracker(
+        this.name,
+        context.getChangeDetectionPolicy(),
+        this.streamStatistics.getChangeTrackerStatistics(),
+        s3Attributes);
+
+    setInputPolicy(context.getInputPolicy());
+    setReadahead(context.getReadahead());
+
+    long fileSize = s3Attributes.getLen();
+    int bufferSize = context.getPrefetchBlockSize();
+
+    this.blockData = new BlockData(fileSize, bufferSize);
+    this.fpos = new FilePosition(fileSize, bufferSize);
+    this.remoteObject = getS3File();
+    this.reader = new S3ARemoteObjectReader(remoteObject);
+
+    this.seekTargetPos = 0;
+  }
+
+  /**
+   * Gets the internal IO statistics.
+   *
+   * @return the internal IO statistics.
+   */
+  @Override
+  public IOStatistics getIOStatistics() {
+    return ioStatistics;
+  }
+
+  /**
+   * Access the input stream statistics.
+   * This is for internal testing and may be removed without warning.
+   * @return the statistics for this input stream
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public S3AInputStreamStatistics getS3AStreamStatistics() {
+    return streamStatistics;
+  }
+
+  /**
+   * Sets the number of bytes to read ahead each time.
+   *
+   * @param readahead the number of bytes to read ahead each time..
+   */
+  @Override
+  public synchronized void setReadahead(Long readahead) {
+    // We support read head by prefetching therefore we ignore the supplied value.
+    if (readahead != null) {
+      Validate.checkNotNegative(readahead, "readahead");
+    }
+  }
+
+  /**
+   * Indicates whether the given {@code capability} is supported by this stream.
+   *
+   * @param capability the capability to check.
+   * @return true if the given {@code capability} is supported by this stream, false otherwise.
+   */
+  @Override
+  public boolean hasCapability(String capability) {
+    return capability.equalsIgnoreCase(StreamCapabilities.IOSTATISTICS)
+        || capability.equalsIgnoreCase(StreamCapabilities.READAHEAD);
+  }
+
+  /**
+   * Set/update the input policy of the stream.
+   * This updates the stream statistics.
+   * @param inputPolicy new input policy.
+   */
+  private void setInputPolicy(S3AInputPolicy inputPolicy) {
+    this.inputPolicy = inputPolicy;
+    streamStatistics.inputPolicySet(inputPolicy.ordinal());
+  }
+
+  /**
+   * Returns the number of bytes that can read from this stream without blocking.
+   */
+  @Override
+  public int available() throws IOException {
+    throwIfClosed();
+
+    if (!ensureCurrentBuffer()) {
+      return 0;
+    }
+
+    return fpos.buffer().remaining();
+  }
+
+  /**
+   * Gets the current position.
+   *
+   * @return the current position.
+   * @throws IOException if there is an IO error during this operation.
+   */
+  public long getPos() throws IOException {
+    throwIfClosed();
+
+    if (fpos.isValid()) {
+      return fpos.absolute();
+    } else {
+      return seekTargetPos;
+    }
+  }
+
+  /**
+   * Moves the current read position so that the next read will occur at {@code pos}.
+   *
+   * @param pos the absolute position to seek to.
+   * @throws IOException if there an error during this operation.
+   *
+   * @throws IllegalArgumentException if pos is outside of the range [0, file size].
+   */
+  public void seek(long pos) throws IOException {
+    throwIfClosed();
+    throwIfInvalidSeek(pos);
+
+    if (!fpos.setAbsolute(pos)) {
+      fpos.invalidate();
+      seekTargetPos = pos;
+    }
+  }
+
+  /**
+   * Ensures that a non-empty valid buffer is available for immediate reading.
+   * It returns true when at least one such buffer is available for reading.
+   * It returns false on reaching the end of the stream.
+   *
+   * @return true if at least one such buffer is available for reading, false otherwise.
+   * @throws IOException if there is an IO error during this operation.
+   */
+  protected abstract boolean ensureCurrentBuffer() throws IOException;
+
+  @Override
+  public int read() throws IOException {
+    throwIfClosed();
+
+    if (remoteObject.size() == 0
+        || seekTargetPos >= remoteObject.size()) {
+      return -1;
+    }
+
+    if (!ensureCurrentBuffer()) {
+      return -1;
+    }
+
+    incrementBytesRead(1);
+
+    return fpos.buffer().get() & 0xff;
+  }
+
+  /**
+   * Reads bytes from this stream and copies them into
+   * the given {@code buffer} starting at the beginning (offset 0).
+   * Returns the number of bytes actually copied in to the given buffer.
+   *
+   * @param buffer the buffer to copy data into.
+   * @return the number of bytes actually copied in to the given buffer.
+   * @throws IOException if there is an IO error during this operation.
+   */
+  @Override
+  public int read(byte[] buffer) throws IOException {
+    return read(buffer, 0, buffer.length);
+  }
+
+  /**
+   * Reads up to {@code len} bytes from this stream and copies them into
+   * the given {@code buffer} starting at the given {@code offset}.
+   * Returns the number of bytes actually copied in to the given buffer.
+   *
+   * @param buffer the buffer to copy data into.
+   * @param offset data is copied starting at this offset.
+   * @param len max number of bytes to copy.
+   * @return the number of bytes actually copied in to the given buffer.
+   * @throws IOException if there is an IO error during this operation.
+   */
+  @Override
+  public int read(byte[] buffer, int offset, int len) throws IOException {
+    throwIfClosed();
+
+    if (len == 0) {
+      return 0;
+    }
+
+    if (remoteObject.size() == 0
+        || seekTargetPos >= remoteObject.size()) {
+      return -1;
+    }
+
+    if (!ensureCurrentBuffer()) {
+      return -1;
+    }
+
+    int numBytesRead = 0;
+    int numBytesRemaining = len;
+
+    while (numBytesRemaining > 0) {
+      if (!ensureCurrentBuffer()) {
+        break;
+      }
+
+      ByteBuffer buf = fpos.buffer();
+      int bytesToRead = Math.min(numBytesRemaining, buf.remaining());
+      buf.get(buffer, offset, bytesToRead);
+      incrementBytesRead(bytesToRead);
+      offset += bytesToRead;
+      numBytesRemaining -= bytesToRead;
+      numBytesRead += bytesToRead;
+    }
+
+    return numBytesRead;
+  }
+
+  protected S3ARemoteObject getFile() {
+    return remoteObject;
+  }
+
+  protected S3ARemoteObjectReader getReader() {
+    return reader;
+  }
+
+  protected S3ObjectAttributes getS3ObjectAttributes() {
+    return s3Attributes;
+  }
+
+  protected FilePosition getFilePosition() {
+    return fpos;
+  }
+
+  protected String getName() {
+    return name;
+  }
+
+  protected boolean isClosed() {
+    return closed;
+  }
+
+  protected long getSeekTargetPos() {
+    return seekTargetPos;
+  }
+
+  protected void setSeekTargetPos(long pos) {
+    seekTargetPos = pos;
+  }
+
+  protected BlockData getBlockData() {
+    return blockData;
+  }
+
+  protected S3AReadOpContext getContext() {
+    return context;
+  }
+
+  private void incrementBytesRead(int bytesRead) {
+    if (bytesRead > 0) {
+      streamStatistics.bytesRead(bytesRead);
+      if (getContext().getStats() != null) {
+        getContext().getStats().incrementBytesRead(bytesRead);
+      }
+      fpos.incrementBytesRead(bytesRead);
+    }
+  }
+
+  protected S3ARemoteObject getS3File() {
+    return new S3ARemoteObject(
+        context,
+        s3Attributes,
+        client,
+        streamStatistics,
+        changeTracker
+    );
+  }
+
+  protected String getOffsetStr(long offset) {
+    int blockNumber = -1;
+
+    if (blockData.isValidOffset(offset)) {
+      blockNumber = blockData.getBlockNumber(offset);
+    }
+
+    return String.format("%d:%d", blockNumber, offset);
+  }
+
+  /**
+   * Closes this stream and releases all acquired resources.
+   *
+   * @throws IOException if there is an IO error during this operation.
+   */
+  @Override
+  public void close() throws IOException {
+    if (closed) {
+      return;
+    }
+    closed = true;
+
+    blockData = null;
+    reader.close();
+    reader = null;
+    remoteObject = null;
+    fpos.invalidate();
+    try {
+      client.close();
+    } finally {
+      streamStatistics.close();
+    }
+    client = null;
+  }
+
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+
+  protected void throwIfClosed() throws IOException {
+    if (closed) {
+      throw new IOException(
+          name + ": " + FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+  }
+
+  protected void throwIfInvalidSeek(long pos) throws EOFException {
+    if (pos < 0) {
+      throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + " " + pos);
+    }
+  }
+
+  // Unsupported functions.
+
+  @Override
+  public void mark(int readlimit) {
+    throw new UnsupportedOperationException("mark not supported");
+  }
+
+  @Override
+  public void reset() {
+    throw new UnsupportedOperationException("reset not supported");
+  }
+
+  @Override
+  public long skip(long n) {
+    throw new UnsupportedOperationException("skip not supported");
+  }
+}

+ 328 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java

@@ -0,0 +1,328 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+import com.amazonaws.services.s3.model.GetObjectRequest;
+import com.amazonaws.services.s3.model.S3Object;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.statistics.DurationTracker;
+
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+
+/**
+ * Encapsulates low level interactions with S3 object on AWS.
+ */
+public class S3ARemoteObject {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3ARemoteObject.class);
+
+  /**
+   * Read-specific operation context.
+   */
+  private final S3AReadOpContext context;
+
+  /**
+   * S3 object attributes.
+   */
+  private final S3ObjectAttributes s3Attributes;
+
+  /**
+   * Callbacks used for interacting with the underlying S3 client.
+   */
+  private final S3AInputStream.InputStreamCallbacks client;
+
+  /**
+   * Used for reporting input stream access statistics.
+   */
+  private final S3AInputStreamStatistics streamStatistics;
+
+  /**
+   * Enforces change tracking related policies.
+   */
+  private final ChangeTracker changeTracker;
+
+  /**
+   * Maps a stream returned by openForRead() to the associated S3 object.
+   * That allows us to close the object when closing the stream.
+   */
+  private Map<InputStream, S3Object> s3Objects;
+
+  /**
+   * uri of the object being read.
+   */
+  private final String uri;
+
+  /**
+   * size of a buffer to create when draining the stream.
+   */
+  private static final int DRAIN_BUFFER_SIZE = 16384;
+
+  /**
+   * Initializes a new instance of the {@code S3ARemoteObject} class.
+   *
+   * @param context read-specific operation context.
+   * @param s3Attributes attributes of the S3 object being read.
+   * @param client callbacks used for interacting with the underlying S3 client.
+   * @param streamStatistics statistics about stream access.
+   * @param changeTracker helps enforce change tracking policy.
+   *
+   * @throws IllegalArgumentException if context is null.
+   * @throws IllegalArgumentException if s3Attributes is null.
+   * @throws IllegalArgumentException if client is null.
+   * @throws IllegalArgumentException if streamStatistics is null.
+   * @throws IllegalArgumentException if changeTracker is null.
+   */
+  public S3ARemoteObject(
+      S3AReadOpContext context,
+      S3ObjectAttributes s3Attributes,
+      S3AInputStream.InputStreamCallbacks client,
+      S3AInputStreamStatistics streamStatistics,
+      ChangeTracker changeTracker) {
+
+    Validate.checkNotNull(context, "context");
+    Validate.checkNotNull(s3Attributes, "s3Attributes");
+    Validate.checkNotNull(client, "client");
+    Validate.checkNotNull(streamStatistics, "streamStatistics");
+    Validate.checkNotNull(changeTracker, "changeTracker");
+
+    this.context = context;
+    this.s3Attributes = s3Attributes;
+    this.client = client;
+    this.streamStatistics = streamStatistics;
+    this.changeTracker = changeTracker;
+    this.s3Objects = new IdentityHashMap<>();
+    this.uri = this.getPath();
+  }
+
+  /**
+   * Gets an instance of {@code Invoker} for interacting with S3 API.
+   *
+   * @return an instance of {@code Invoker} for interacting with S3 API.
+   */
+  public Invoker getReadInvoker() {
+    return context.getReadInvoker();
+  }
+
+  /**
+   * Gets an instance of {@code S3AInputStreamStatistics} used for reporting access metrics.
+   *
+   * @return an instance of {@code S3AInputStreamStatistics} used for reporting access metrics.
+   */
+  public S3AInputStreamStatistics getStatistics() {
+    return streamStatistics;
+  }
+
+  /**
+   * Gets the path of this file.
+   *
+   * @return the path of this file.
+   */
+  public String getPath() {
+    return getPath(s3Attributes);
+  }
+
+  /**
+   * Gets the path corresponding to the given s3Attributes.
+   *
+   * @param s3Attributes attributes of an S3 object.
+   * @return the path corresponding to the given s3Attributes.
+   */
+  public static String getPath(S3ObjectAttributes s3Attributes) {
+    return String.format("s3a://%s/%s", s3Attributes.getBucket(),
+        s3Attributes.getKey());
+  }
+
+  /**
+   * Gets the size of this file.
+   * Its value is cached once obtained from AWS.
+   *
+   * @return the size of this file.
+   */
+  public long size() {
+    return s3Attributes.getLen();
+  }
+
+  /**
+   * Opens a section of the file for reading.
+   *
+   * @param offset Start offset (0 based) of the section to read.
+   * @param size Size of the section to read.
+   * @return an {@code InputStream} corresponding to the given section of this file.
+   *
+   * @throws IOException if there is an error opening this file section for reading.
+   * @throws IllegalArgumentException if offset is negative.
+   * @throws IllegalArgumentException if offset is greater than or equal to file size.
+   * @throws IllegalArgumentException if size is greater than the remaining bytes.
+   */
+  public InputStream openForRead(long offset, int size) throws IOException {
+    Validate.checkNotNegative(offset, "offset");
+    Validate.checkLessOrEqual(offset, "offset", size(), "size()");
+    Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset");
+
+    streamStatistics.streamOpened();
+    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);
+    DurationTracker tracker = streamStatistics.initiateGetRequest();
+    S3Object object = null;
+
+    try {
+      object = Invoker.once(operation, uri, () -> client.getObject(request));
+    } catch (IOException e) {
+      tracker.failed();
+      throw e;
+    } finally {
+      tracker.close();
+    }
+
+    changeTracker.processResponse(object, operation, offset);
+    InputStream stream = object.getObjectContent();
+    synchronized (s3Objects) {
+      s3Objects.put(stream, object);
+    }
+
+    return stream;
+  }
+
+  void close(InputStream inputStream, int numRemainingBytes) {
+    S3Object obj;
+    synchronized (s3Objects) {
+      obj = s3Objects.get(inputStream);
+      if (obj == null) {
+        throw new IllegalArgumentException("inputStream not found");
+      }
+      s3Objects.remove(inputStream);
+    }
+
+    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));
+    }
+  }
+
+  /**
+   * drain the stream. This method is intended to be
+   * used directly or asynchronously, and measures the
+   * duration of the operation in the stream statistics.
+   *
+   * @param shouldAbort   force an abort; used if explicitly requested.
+   * @param reason        reason for stream being closed; used in messages
+   * @param remaining     remaining bytes
+   * @param requestObject http request object;
+   * @param inputStream   stream to close.
+   * @return was the stream aborted?
+   */
+  private boolean drain(
+      final boolean shouldAbort,
+      final String reason,
+      final long remaining,
+      final S3Object requestObject,
+      final InputStream inputStream) {
+
+    try {
+      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
+      return shouldAbort;
+    }
+  }
+
+  /**
+   * Drain or abort the inner stream.
+   * Exceptions are swallowed.
+   * If a close() is attempted and fails, the operation escalates to
+   * an abort.
+   *
+   * @param shouldAbort   force an abort; used if explicitly requested.
+   * @param reason        reason for stream being closed; used in messages
+   * @param remaining     remaining bytes
+   * @param requestObject http request object
+   * @param inputStream   stream to close.
+   * @return was the stream aborted?
+   */
+  private boolean drainOrAbortHttpStream(
+      boolean shouldAbort,
+      final String reason,
+      final long remaining,
+      final S3Object requestObject,
+      final InputStream inputStream) {
+
+    if (!shouldAbort && remaining > 0) {
+      try {
+        long drained = 0;
+        byte[] buffer = new byte[DRAIN_BUFFER_SIZE];
+        while (true) {
+          final int count = inputStream.read(buffer);
+          if (count < 0) {
+            // no more data is left
+            break;
+          }
+          drained += count;
+        }
+        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);
+        shouldAbort = true;
+      }
+    }
+    cleanupWithLogger(LOG, inputStream);
+    cleanupWithLogger(LOG, requestObject);
+    streamStatistics.streamClose(shouldAbort, remaining);
+
+    LOG.debug("Stream {} {}: {}; remaining={}", uri,
+        (shouldAbort ? "aborted" : "closed"), reason,
+        remaining);
+    return shouldAbort;
+  }
+}

+ 175 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java

@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.SocketTimeoutException;
+import java.nio.ByteBuffer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.impl.prefetch.Validate;
+import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_REMOTE_BLOCK_READ;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
+
+/**
+ * Provides functionality to read S3 file one block at a time.
+ */
+public class S3ARemoteObjectReader implements Closeable {
+
+  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 S3ARemoteObject remoteObject;
+
+  /** Set to true by close(). */
+  private volatile boolean closed;
+
+  private final S3AInputStreamStatistics streamStatistics;
+
+  /**
+   * Constructs an instance of {@link S3ARemoteObjectReader}.
+   *
+   * @param remoteObject The S3 file to read.
+   *
+   * @throws IllegalArgumentException if remoteObject is null.
+   */
+  public S3ARemoteObjectReader(S3ARemoteObject remoteObject) {
+    Validate.checkNotNull(remoteObject, "remoteObject");
+
+    this.remoteObject = remoteObject;
+    this.streamStatistics = this.remoteObject.getStatistics();
+  }
+
+  /**
+   * Stars reading at {@code offset} and reads upto {@code size} bytes into {@code buffer}.
+   *
+   * @param buffer the buffer into which data is returned
+   * @param offset the absolute offset into the underlying file where reading starts.
+   * @param size the number of bytes to be read.
+   *
+   * @return number of bytes actually read.
+   * @throws IOException if there is an error reading from the file.
+   *
+   * @throws IllegalArgumentException if buffer is null.
+   * @throws IllegalArgumentException if offset is outside of the range [0, file size].
+   * @throws IllegalArgumentException if size is zero or negative.
+   */
+  public int read(ByteBuffer buffer, long offset, int size) throws IOException {
+    Validate.checkNotNull(buffer, "buffer");
+    Validate.checkWithinRange(offset, "offset", 0, this.remoteObject.size());
+    Validate.checkPositiveInteger(size, "size");
+
+    if (this.closed) {
+      return -1;
+    }
+
+    int reqSize = (int) Math.min(size, this.remoteObject.size() - offset);
+    return readOneBlockWithRetries(buffer, offset, reqSize);
+  }
+
+  @Override
+  public void close() {
+    this.closed = true;
+  }
+
+  private int readOneBlockWithRetries(ByteBuffer buffer, long offset, int size)
+      throws IOException {
+
+    this.streamStatistics.readOperationStarted(offset, size);
+    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.remoteObject.getStatistics()
+        .readOperationCompleted(size, numBytesRead);
+
+    if (invokerResponse < 0) {
+      return invokerResponse;
+    } else {
+      return numBytesRead;
+    }
+  }
+
+  private void readOneBlock(ByteBuffer buffer, long offset, int size)
+      throws IOException {
+    int readSize = Math.min(size, buffer.remaining());
+    if (readSize == 0) {
+      return;
+    }
+
+    InputStream inputStream = remoteObject.openForRead(offset, readSize);
+    int numRemainingBytes = readSize;
+    byte[] bytes = new byte[READ_BUFFER_SIZE];
+
+    int numBytesToRead;
+    int numBytes;
+
+    try {
+      do {
+        numBytesToRead = Math.min(READ_BUFFER_SIZE, numRemainingBytes);
+        numBytes = inputStream.read(bytes, 0, numBytesToRead);
+        if (numBytes < 0) {
+          String message = String.format(
+              "Unexpected end of stream: buffer[%d], readSize = %d, numRemainingBytes = %d",
+              buffer.capacity(), readSize, numRemainingBytes);
+          throw new EOFException(message);
+        }
+
+        if (numBytes > 0) {
+          buffer.put(bytes, 0, numBytes);
+          numRemainingBytes -= numBytes;
+        }
+      }
+      while (!this.closed && (numRemainingBytes > 0));
+    } finally {
+      remoteObject.close(inputStream, numRemainingBytes);
+    }
+  }
+}

+ 29 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/package-info.java

@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/**
+ * 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.prefetch;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 2 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a.statistics;
 
+import org.apache.hadoop.fs.impl.prefetch.PrefetchingStatistics;
 import org.apache.hadoop.fs.statistics.DurationTracker;
 
 /**
@@ -26,7 +27,7 @@ import org.apache.hadoop.fs.statistics.DurationTracker;
  * It also contains getters for tests.
  */
 public interface S3AInputStreamStatistics extends AutoCloseable,
-    S3AStatisticInterface {
+    S3AStatisticInterface, PrefetchingStatistics {
 
   /**
    * Seek backwards, incrementing the seek and backward seek counters.

+ 36 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java

@@ -221,6 +221,41 @@ public final class EmptyS3AStatisticsContext implements S3AStatisticsContext {
 
     }
 
+    @Override
+    public DurationTracker prefetchOperationStarted() {
+      return stubDurationTracker();
+    }
+
+    @Override
+    public void prefetchOperationCompleted() {
+
+    }
+
+    @Override
+    public void blockAddedToFileCache() {
+
+    }
+
+    @Override
+    public void blockRemovedFromFileCache() {
+
+    }
+
+    @Override
+    public void executorAcquired(Duration timeInQueue) {
+
+    }
+
+    @Override
+    public void memoryAllocated(int size) {
+
+    }
+
+    @Override
+    public void memoryFreed(int size) {
+
+    }
+
     /**
      * Return an IO statistics instance.
      * @return an empty IO statistics instance.
@@ -354,6 +389,7 @@ public final class EmptyS3AStatisticsContext implements S3AStatisticsContext {
     public DurationTracker initiateInnerStreamClose(final boolean abort) {
       return stubDurationTracker();
     }
+
   }
 
   /**

+ 24 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md

@@ -47,6 +47,7 @@ full details.
 * [Auditing](./auditing.html).
 * [Auditing Architecture](./auditing_architecture.html).
 * [Testing](./testing.html)
+* [Prefetching](./prefetching.html)
 * [Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html)
 
 ## <a name="overview"></a> Overview
@@ -1091,6 +1092,29 @@ options are covered in [Testing](./testing.md).
   </description>
 </property>
 
+<property>
+  <name>fs.s3a.prefetch.enabled</name>
+  <value>false</value>
+  <description>
+    Enables prefetching and caching when reading from input stream.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.prefetch.block.size</name>
+  <value>8MB</value>
+  <description>
+      The size of a single prefetched block of data.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.prefetch.block.count</name>
+  <value>8</value>
+  <description>
+      Maximum number of blocks prefetched concurrently at any given time.
+  </description>
+</property>
 ```
 
 ## <a name="retry_and_recovery"></a>Retry and Recovery

+ 192 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md

@@ -0,0 +1,192 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# S3A Prefetching
+
+This document explains the `S3PrefetchingInputStream` and the various components it uses.
+
+This input stream implements prefetching and caching to improve read performance of the input
+stream.
+A high level overview of this feature was published in
+[Pinterest Engineering's blog post titled "Improving efficiency and reducing runtime using S3 read optimization"](https://medium.com/pinterest-engineering/improving-efficiency-and-reducing-runtime-using-s3-read-optimization-b31da4b60fa0).
+
+With prefetching, the input stream divides the remote file into blocks of a fixed size, associates
+buffers to these blocks and then reads data into these buffers asynchronously.
+It also potentially caches these blocks.
+
+### Basic Concepts
+
+* **Remote File**: A binary blob of data stored on some storage device.
+* **Block File**: Local file containing a block of the remote file.
+* **Block**: A file is divided into a number of blocks.
+The size of the first n-1 blocks is same, and the size of the last block may be same or smaller.
+* **Block based reading**: The granularity of read is one block.
+That is, either an entire block is read and returned or none at all.
+Multiple blocks may be read in parallel.
+
+### Configuring the stream
+
+|Property    |Meaning    |Default    |
+|---|---|---|
+|`fs.s3a.prefetch.enabled`    |Enable the prefetch input stream    |`false` |
+|`fs.s3a.prefetch.block.size`    |Size of a block    |`8M`    |
+|`fs.s3a.prefetch.block.count`    |Number of blocks to prefetch    |`8`    |
+
+### Key Components
+
+`S3PrefetchingInputStream` - When prefetching is enabled, S3AFileSystem will return an instance of
+this class as the input stream.
+Depending on the remote file size, it will either use
+the `S3InMemoryInputStream` or the `S3CachingInputStream` as the underlying input stream.
+
+`S3InMemoryInputStream` - Underlying input stream used when the remote file size < configured block
+size.
+Will read the entire remote file into memory.
+
+`S3CachingInputStream` - Underlying input stream used when remote file size > configured block size.
+Uses asynchronous prefetching of blocks and caching to improve performance.
+
+`BlockData` - Holds information about the blocks in a remote file, such as:
+
+* Number of blocks in the remote file
+* Block size
+* State of each block (initially all blocks have state *NOT_READY*).
+Other states are: Queued, Ready, Cached.
+
+`BufferData` - Holds the buffer and additional information about it such as:
+
+* The block number this buffer is for
+* State of the buffer (Unknown, Blank, Prefetching, Caching, Ready, Done).
+Initial state of a buffer is blank.
+
+`CachingBlockManager` - Implements reading data into the buffer, prefetching and caching.
+
+`BufferPool` - Manages a fixed sized pool of buffers.
+It's used by `CachingBlockManager` to acquire buffers.
+
+`S3File` - Implements operations to interact with S3 such as opening and closing the input stream to
+the remote file in S3.
+
+`S3Reader` - Implements reading from the stream opened by `S3File`.
+Reads from this input stream in blocks of 64KB.
+
+`FilePosition` - Provides functionality related to tracking the position in the file.
+Also gives access to the current buffer in use.
+
+`SingleFilePerBlockCache` - Responsible for caching blocks to the local file system.
+Each cache block is stored on the local disk as a separate block file.
+
+### Operation
+
+#### S3InMemoryInputStream
+
+For a remote file with size 5MB, and block size = 8MB, since file size is less than the block size,
+the `S3InMemoryInputStream` will be used.
+
+If the caller makes the following read calls:
+
+```
+in.read(buffer, 0, 3MB);
+in.read(buffer, 0, 2MB);
+```
+
+When the first read is issued, there is no buffer in use yet.
+The `S3InMemoryInputStream` gets the data in this remote file by calling the `ensureCurrentBuffer()`
+method, which ensures that a buffer with data is available to be read from.
+
+The `ensureCurrentBuffer()` then:
+
+* Reads data into a buffer by calling `S3Reader.read(ByteBuffer buffer, long offset, int size)`.
+* `S3Reader` uses `S3File` to open an input stream to the remote file in S3 by making
+  a `getObject()` request with range as `(0, filesize)`.
+* The `S3Reader` reads the entire remote file into the provided buffer, and once reading is complete
+  closes the S3 stream and frees all underlying resources.
+* Now the entire remote file is in a buffer, set this data in `FilePosition` so it can be accessed
+  by the input stream.
+
+The read operation now just gets the required bytes from the buffer in `FilePosition`.
+
+When the second read is issued, there is already a valid buffer which can be used.
+Don't do anything else, just read the required bytes from this buffer.
+
+#### S3CachingInputStream
+
+If there is a remote file with size 40MB and block size = 8MB, the `S3CachingInputStream` will be
+used.
+
+##### Sequential Reads
+
+If the caller makes the following calls:
+
+```
+in.read(buffer, 0, 5MB)
+in.read(buffer, 0, 8MB)
+```
+
+For the first read call, there is no valid buffer yet.
+`ensureCurrentBuffer()` is called, and for the first `read()`, prefetch count is set as 1.
+
+The current block (block 0) is read synchronously, while the blocks to be prefetched (block 1) is
+read asynchronously.
+
+The `CachingBlockManager` is responsible for getting buffers from the buffer pool and reading data
+into them. This process of acquiring the buffer pool works as follows:
+
+* The buffer pool keeps a map of allocated buffers and a pool of available buffers.
+The size of this pool is = prefetch block count + 1.
+If the prefetch block count is 8, the buffer pool has a size of 9.
+* If the pool is not yet at capacity, create a new buffer and add it to the pool.
+* If it's at capacity, check if any buffers with state = done can be released.
+Releasing a buffer means removing it from allocated and returning it back to the pool of available
+buffers.
+* If there are no buffers with state = done currently then nothing will be released, so retry the
+  above step at a fixed interval a few times till a buffer becomes available.
+* If after multiple retries there are still no available buffers, release a buffer in the ready state.
+The buffer for the block furthest from the current block is released.
+
+Once a buffer has been acquired by `CachingBlockManager`, if the buffer is in a *READY* state, it is
+returned.
+This means that data was already read into this buffer asynchronously by a prefetch.
+If it's state is *BLANK* then data is read into it using
+`S3Reader.read(ByteBuffer buffer, long offset, int size).`
+
+For the second read call, `in.read(buffer, 0, 8MB)`, since the block sizes are of 8MB and only 5MB
+of block 0 has been read so far, 3MB of the required data will be read from the current block 0.
+Once all data has been read from this block, `S3CachingInputStream` requests the next block (
+block 1), which will already have been prefetched and so it can just start reading from it.
+Also, while reading from block 1 it will also issue prefetch requests for the next blocks.
+The number of blocks to be prefetched is determined by `fs.s3a.prefetch.block.count`.
+
+##### Random Reads
+
+If the caller makes the following calls:
+
+```
+in.read(buffer, 0, 5MB)
+in.seek(10MB)
+in.read(buffer, 0, 4MB)
+in.seek(2MB)
+in.read(buffer, 0, 4MB)
+```
+
+The `CachingInputStream` also caches prefetched blocks.
+This happens when a `seek()` is issued for outside the current block and the current block still has
+not been fully read.
+
+For the above read sequence, when the `seek(10MB)` call is issued, block 0 has not been read
+completely so cache it as the caller will probably want to read from it again.
+
+When `seek(2MB)` is called, the position is back inside block 0.
+The next read can now be satisfied from the locally cached block file, which is typically orders of
+magnitude faster than a network based read.

+ 189 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java

@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.net.URI;
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.StoreStatisticNames;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_SIZE;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+
+/**
+ * Test the prefetching input stream, validates that the underlying S3ACachingInputStream and
+ * S3AInMemoryInputStream are working as expected.
+ */
+public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
+
+  public ITestS3APrefetchingInputStream() {
+    super(true);
+  }
+
+  private static final Logger LOG =
+      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 S3ACachingInputStream is used
+  private Path largeFile;
+  private FileSystem largeFileFS;
+  private int numBlocks;
+  private int blockSize;
+  private long largeFileSize;
+  // Size should be < block size so S3AInMemoryInputStream is used
+  private static final int SMALL_FILE_SIZE = S_1K * 16;
+
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
+    conf.setBoolean(PREFETCH_ENABLED_KEY, true);
+    return conf;
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    cleanupWithLogger(LOG, largeFileFS);
+    largeFileFS = null;
+  }
+
+  private void openFS() throws Exception {
+    Configuration conf = getConfiguration();
+
+    largeFile = new Path(DEFAULT_CSVTEST_FILE);
+    blockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE);
+    largeFileFS = new S3AFileSystem();
+    largeFileFS.initialize(new URI(DEFAULT_CSVTEST_FILE), getConfiguration());
+    FileStatus fileStatus = largeFileFS.getFileStatus(largeFile);
+    largeFileSize = fileStatus.getLen();
+    numBlocks = calculateNumBlocks(largeFileSize, blockSize);
+  }
+
+  private static int calculateNumBlocks(long largeFileSize, int blockSize) {
+    if (largeFileSize == 0) {
+      return 0;
+    } else {
+      return ((int) (largeFileSize / blockSize)) + (largeFileSize % blockSize > 0 ? 1 : 0);
+    }
+  }
+
+  @Test
+  public void testReadLargeFileFully() throws Throwable {
+    describe("read a large file fully, uses S3ACachingInputStream");
+    IOStatistics ioStats;
+    openFS();
+
+    try (FSDataInputStream in = largeFileFS.open(largeFile)) {
+      ioStats = in.getIOStatistics();
+
+      byte[] buffer = new byte[S_1M * 10];
+      long bytesRead = 0;
+
+      while (bytesRead < largeFileSize) {
+        in.readFully(buffer, 0, (int) Math.min(buffer.length, largeFileSize - bytesRead));
+        bytesRead += buffer.length;
+        // Blocks are fully read, no blocks should be cached
+        verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE,
+            0);
+      }
+
+      // Assert that first block is read synchronously, following blocks are prefetched
+      verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS,
+          numBlocks - 1);
+      verifyStatisticCounterValue(ioStats, StoreStatisticNames.ACTION_HTTP_GET_REQUEST, numBlocks);
+      verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_OPENED, numBlocks);
+    }
+    // Verify that once stream is closed, all memory is freed
+    verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, 0);
+  }
+
+  @Test
+  public void testRandomReadLargeFile() throws Throwable {
+    describe("random read on a large file, uses S3ACachingInputStream");
+    IOStatistics ioStats;
+    openFS();
+
+    try (FSDataInputStream in = largeFileFS.open(largeFile)) {
+      ioStats = in.getIOStatistics();
+
+      byte[] buffer = new byte[blockSize];
+
+      // Don't read the block completely so it gets cached on seek
+      in.read(buffer, 0, blockSize - S_1K * 10);
+      in.seek(blockSize + S_1K * 10);
+      // Backwards seek, will use cached block
+      in.seek(S_1K * 5);
+      in.read();
+
+      verifyStatisticCounterValue(ioStats, StoreStatisticNames.ACTION_HTTP_GET_REQUEST, 2);
+      verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_OPENED, 2);
+      verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS, 1);
+      // block 0 is cached when we seek to block 1, block 1 is cached as it is being prefetched
+      // when we seek out of block 0, see cancelPrefetches()
+      verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE, 2);
+    }
+    verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE, 0);
+    verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, 0);
+  }
+
+  @Test
+  public void testRandomReadSmallFile() throws Throwable {
+    describe("random read on a small file, uses S3AInMemoryInputStream");
+
+    byte[] data = ContractTestUtils.dataset(SMALL_FILE_SIZE, 'a', 26);
+    Path smallFile = path("randomReadSmallFile");
+    ContractTestUtils.writeDataset(getFileSystem(), smallFile, data, data.length, 16, true);
+
+    try (FSDataInputStream in = getFileSystem().open(smallFile)) {
+      IOStatistics ioStats = in.getIOStatistics();
+
+      byte[] buffer = new byte[SMALL_FILE_SIZE];
+
+      in.read(buffer, 0, S_1K * 4);
+      in.seek(S_1K * 12);
+      in.read(buffer, 0, S_1K * 4);
+
+      verifyStatisticCounterValue(ioStats, StoreStatisticNames.ACTION_HTTP_GET_REQUEST, 1);
+      verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_OPENED, 1);
+      verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS, 0);
+      // The buffer pool is not used
+      verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, 0);
+    }
+  }
+
+}

+ 12 - 5
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java

@@ -31,6 +31,8 @@ import org.apache.hadoop.fs.statistics.IOStatisticAssertions;
 import org.apache.hadoop.fs.statistics.StreamStatisticNames;
 
 import static org.apache.hadoop.fs.s3a.Constants.ALLOW_REQUESTER_PAYS;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_DEFAULT;
+import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
@@ -78,11 +80,16 @@ public class ITestS3ARequesterPays extends AbstractS3ATestBase {
       inputStream.seek(0);
       inputStream.readByte();
 
-      // Verify > 1 call was made, so we're sure it is correctly configured for each request
-      IOStatisticAssertions
-          .assertThatStatisticCounter(inputStream.getIOStatistics(),
-              StreamStatisticNames.STREAM_READ_OPENED)
-          .isGreaterThan(1);
+      if (conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT)) {
+        // For S3APrefetchingInputStream, verify a call was made
+        IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(),
+            StreamStatisticNames.STREAM_READ_OPENED).isEqualTo(1);
+      } else {
+        // For S3AInputStream, verify > 1 call was made,
+        // so we're sure it is correctly configured for each request
+        IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(),
+            StreamStatisticNames.STREAM_READ_OPENED).isGreaterThan(1);
+      }
 
       // Check list calls work without error
       fs.listFiles(requesterPaysPath.getParent(), false);

+ 10 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a;
 
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
 import org.apache.hadoop.fs.statistics.IOStatistics;
@@ -33,6 +34,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES;
 import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE;
 import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_TOTAL_BYTES;
@@ -72,6 +74,7 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
     IOStatisticsSnapshot iostats = new IOStatisticsSnapshot();
     // Open file, read half the data, and then call unbuffer
     try (FSDataInputStream inputStream = getFileSystem().open(dest)) {
+      skipIfCannotUnbuffer(inputStream);
       assertTrue(inputStream.getWrappedStream() instanceof S3AInputStream);
       int bytesToRead = 8;
       readAndAssertBytesRead(inputStream, bytesToRead);
@@ -138,6 +141,7 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
     Object streamStatsStr;
     try {
       inputStream = fs.open(dest);
+      skipIfCannotUnbuffer(inputStream);
       streamStatsStr = demandStringifyIOStatisticsSource(inputStream);
 
       LOG.info("initial stream statistics {}", streamStatsStr);
@@ -192,6 +196,12 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
     return ((S3AInputStream) inputStream.getWrappedStream()).isObjectStreamOpen();
   }
 
+  private void skipIfCannotUnbuffer(FSDataInputStream inputStream) {
+    if (!inputStream.hasCapability(StreamCapabilities.UNBUFFER)) {
+      skip("input stream does not support unbuffer");
+    }
+  }
+
   /**
    * Read the specified number of bytes from the given
    * {@link FSDataInputStream} and assert that

+ 12 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.s3a.impl.ContextAccessors;
 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.statistics.BlockOutputStreamStatistics;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
 import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
@@ -1461,16 +1462,24 @@ public final class S3ATestUtils {
     }
   }
 
-
   /**
    * Get the input stream statistics of an input stream.
    * Raises an exception if the inner stream is not an S3A input stream
+   * or prefetching input stream
    * @param in wrapper
    * @return the statistics for the inner stream
    */
   public static S3AInputStreamStatistics getInputStreamStatistics(
-          FSDataInputStream in) {
-    return getS3AInputStream(in).getS3AStreamStatistics();
+      FSDataInputStream in) {
+
+    InputStream inner = in.getWrappedStream();
+    if (inner instanceof S3AInputStream) {
+      return ((S3AInputStream) inner).getS3AStreamStatistics();
+    } else if (inner instanceof S3APrefetchingInputStream) {
+      return ((S3APrefetchingInputStream) inner).getS3AStreamStatistics();
+    } else {
+      throw new AssertionError("Not an S3AInputStream or S3APrefetchingInputStream: " + inner);
+    }
   }
 
   /**

+ 2 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java

@@ -34,8 +34,8 @@ import static org.junit.Assert.assertEquals;
 
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.Mockito.atLeast;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -74,6 +74,6 @@ public class TestS3AUnbuffer extends AbstractS3AMockTest {
     stream.unbuffer();
 
     // Verify that unbuffer closed the object stream
-    verify(objectStream, times(1)).close();
+    verify(objectStream, atLeast(1)).close();
   }
 }

+ 114 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java

@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.CompletableFuture;
+
+import com.amazonaws.services.s3.model.GetObjectRequest;
+import com.amazonaws.services.s3.model.S3Object;
+
+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;
+
+/**
+ * A mock s3 file with some fault injection.
+ */
+class MockS3ARemoteObject extends S3ARemoteObject {
+
+  private byte[] contents;
+
+  // If true, throws IOException on open request just once.
+  // That allows test code to validate behavior related to retries.
+  private boolean throwExceptionOnOpen;
+
+  private static final String BUCKET = "bucket";
+
+  private static final String KEY = "key";
+
+  MockS3ARemoteObject(int size) {
+    this(size, false);
+  }
+
+  MockS3ARemoteObject(int size, boolean throwExceptionOnOpen) {
+    super(
+        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;
+    this.contents = new byte[size];
+    for (int b = 0; b < size; b++) {
+      this.contents[b] = byteAtOffset(b);
+    }
+  }
+
+  @Override
+  public InputStream openForRead(long offset, int size) throws IOException {
+    Validate.checkLessOrEqual(offset, "offset", size(), "size()");
+    Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset");
+
+    if (throwExceptionOnOpen) {
+      throwExceptionOnOpen = false;
+      throw new IOException("Throwing because throwExceptionOnOpen is true ");
+    }
+    int bufSize = (int) Math.min(size, size() - offset);
+    return new ByteArrayInputStream(contents, (int) offset, bufSize);
+  }
+
+  @Override
+  public void close(InputStream inputStream, int numRemainingBytes) {
+    // do nothing since we do not use a real S3 stream.
+  }
+
+  public static byte byteAtOffset(int offset) {
+    return (byte) (offset % 128);
+  }
+
+  public static S3AInputStream.InputStreamCallbacks createClient(String bucketName) {
+    return new S3AInputStream.InputStreamCallbacks() {
+      @Override
+      public S3Object getObject(GetObjectRequest request) {
+        return null;
+      }
+
+      @Override
+      public <T> CompletableFuture<T> submit(CallableRaisingIOE<T> operation) {
+        return null;
+      }
+
+      @Override
+      public GetObjectRequest newGetRequest(String key) {
+        return new GetObjectRequest(bucketName, key);
+      }
+
+      @Override
+      public void close() {
+      }
+    };
+  }
+}

+ 413 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java

@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import com.amazonaws.services.s3.model.GetObjectRequest;
+import com.amazonaws.services.s3.model.ObjectMetadata;
+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.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;
+import org.apache.hadoop.fs.s3a.S3AInputPolicy;
+import org.apache.hadoop.fs.s3a.S3AInputStream;
+import org.apache.hadoop.fs.s3a.S3AReadOpContext;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.VectoredIOContext;
+import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
+import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
+import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker;
+import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
+
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatisticsStore;
+
+/**
+ * Provides 'fake' implementations of S3ARemoteInputStream variants.
+ *
+ * These implementations avoid accessing the following real resources:
+ * -- S3 store
+ * -- local filesystem
+ *
+ * This arrangement allows thorough multi-threaded testing of those
+ * implementations without accessing external resources. It also helps
+ * avoid test flakiness introduced by external factors.
+ */
+public final class S3APrefetchFakes {
+
+  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,
+          ChangeDetectionPolicy.Source.None,
+          false);
+
+  public static S3AFileStatus createFileStatus(String key, long fileSize) {
+    org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
+    long blockSize = fileSize;
+    return new S3AFileStatus(
+        fileSize, MODIFICATION_TIME, path, blockSize, OWNER, E_TAG, VERSION_ID);
+  }
+
+  public static S3ObjectAttributes createObjectAttributes(
+      String bucket,
+      String key,
+      long fileSize) {
+
+    org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
+    String encryptionKey = "";
+
+    return new S3ObjectAttributes(
+        bucket,
+        path,
+        key,
+        S3AEncryptionMethods.NONE,
+        encryptionKey,
+        E_TAG,
+        VERSION_ID,
+        fileSize);
+  }
+
+  public static S3AReadOpContext createReadContext(
+      ExecutorServiceFuturePool futurePool,
+      String key,
+      int fileSize,
+      int prefetchBlockSize,
+      int prefetchBlockCount) {
+
+    S3AFileStatus fileStatus = createFileStatus(key, fileSize);
+    org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
+    FileSystem.Statistics statistics = new FileSystem.Statistics("s3a");
+    S3AStatisticsContext statisticsContext = new EmptyS3AStatisticsContext();
+    RetryPolicy retryPolicy =
+        RetryPolicies.retryUpToMaximumCountWithFixedSleep(3, 10,
+            TimeUnit.MILLISECONDS);
+
+    return new S3AReadOpContext(
+        path,
+        new Invoker(retryPolicy, Invoker.LOG_EVENT),
+        statistics,
+        statisticsContext,
+        fileStatus,
+        new VectoredIOContext()
+            .setMinSeekForVectoredReads(1)
+            .setMaxReadSizeForVectoredReads(1)
+            .build(),
+        emptyStatisticsStore(),
+        futurePool,
+        prefetchBlockSize,
+        prefetchBlockCount)
+        .withChangeDetectionPolicy(
+            ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None,
+                ChangeDetectionPolicy.Source.ETag, false))
+        .withInputPolicy(S3AInputPolicy.Normal);
+  }
+
+  public static URI createUri(String bucket, String key) {
+    return URI.create(String.format("s3a://%s/%s", bucket, key));
+  }
+
+  public static ChangeTracker createChangeTracker(
+      String bucket,
+      String key,
+      long fileSize) {
+
+    return new ChangeTracker(
+        createUri(bucket, key).toString(),
+        CHANGE_POLICY,
+        new CountingChangeTracker(),
+        createObjectAttributes(bucket, key, fileSize));
+  }
+
+  public static S3ObjectInputStream createS3ObjectInputStream(byte[] buffer) {
+    return new S3ObjectInputStream(new ByteArrayInputStream(buffer), null);
+  }
+
+  public static S3AInputStream.InputStreamCallbacks createInputStreamCallbacks(
+      String bucket,
+      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;
+      }
+    };
+
+    return new S3AInputStream.InputStreamCallbacks() {
+      @Override
+      public S3Object getObject(GetObjectRequest request) {
+        return object;
+      }
+
+      @Override
+      public <T> CompletableFuture<T> submit(CallableRaisingIOE<T> operation) {
+        return null;
+      }
+
+      @Override
+      public GetObjectRequest newGetRequest(String key) {
+        return new GetObjectRequest(bucket, key);
+      }
+
+      @Override
+      public void close() {
+      }
+    };
+  }
+
+
+  public static S3ARemoteInputStream createInputStream(
+      Class<? extends S3ARemoteInputStream> clazz,
+      ExecutorServiceFuturePool futurePool,
+      String bucket,
+      String key,
+      int fileSize,
+      int prefetchBlockSize,
+      int prefetchBlockCount) {
+
+    org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
+
+    S3AFileStatus fileStatus = createFileStatus(key, fileSize);
+    S3ObjectAttributes s3ObjectAttributes =
+        createObjectAttributes(bucket, key, fileSize);
+    S3AReadOpContext s3AReadOpContext = createReadContext(
+        futurePool,
+        key,
+        fileSize,
+        prefetchBlockSize,
+        prefetchBlockCount);
+
+    S3AInputStream.InputStreamCallbacks callbacks =
+        createInputStreamCallbacks(bucket, key);
+    S3AInputStreamStatistics stats =
+        s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics();
+
+    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 FakeS3AInMemoryInputStream createS3InMemoryInputStream(
+      ExecutorServiceFuturePool futurePool,
+      String bucket,
+      String key,
+      int fileSize) {
+
+    return (FakeS3AInMemoryInputStream) createInputStream(
+        FakeS3AInMemoryInputStream.class, futurePool, bucket, key, fileSize, 1,
+        1);
+  }
+
+  public static FakeS3ACachingInputStream createS3CachingInputStream(
+      ExecutorServiceFuturePool futurePool,
+      String bucket,
+      String key,
+      int fileSize,
+      int prefetchBlockSize,
+      int prefetchBlockCount) {
+
+    return (FakeS3ACachingInputStream) createInputStream(
+        FakeS3ACachingInputStream.class,
+        futurePool,
+        bucket,
+        key,
+        fileSize,
+        prefetchBlockSize,
+        prefetchBlockCount);
+  }
+
+  public static class FakeS3AInMemoryInputStream
+      extends S3AInMemoryInputStream {
+
+    public FakeS3AInMemoryInputStream(
+        S3AReadOpContext context,
+        S3ObjectAttributes s3Attributes,
+        S3AInputStream.InputStreamCallbacks client,
+        S3AInputStreamStatistics streamStatistics) {
+      super(context, s3Attributes, client, streamStatistics);
+    }
+
+    @Override
+    protected S3ARemoteObject getS3File() {
+      randomDelay(200);
+      return new MockS3ARemoteObject(
+          (int) this.getS3ObjectAttributes().getLen(), false);
+    }
+  }
+
+  public static class FakeS3FilePerBlockCache extends SingleFilePerBlockCache {
+
+    private final Map<Path, byte[]> files;
+
+    private final int readDelay;
+
+    private final int writeDelay;
+
+    public FakeS3FilePerBlockCache(int readDelay, int writeDelay) {
+      super(new EmptyS3AStatisticsContext().newInputStreamStatistics());
+      this.files = new ConcurrentHashMap<>();
+      this.readDelay = readDelay;
+      this.writeDelay = writeDelay;
+    }
+
+    @Override
+    protected int readFile(Path path, ByteBuffer buffer) {
+      byte[] source = this.files.get(path);
+      randomDelay(this.readDelay);
+      buffer.put(source);
+      return source.length;
+    }
+
+    @Override
+    protected void writeFile(Path path, ByteBuffer buffer) throws IOException {
+      Validate.checkPositiveInteger(buffer.limit(), "buffer.limit()");
+      byte[] dest = new byte[buffer.limit()];
+      randomDelay(this.writeDelay);
+      buffer.rewind();
+      buffer.get(dest);
+      this.files.put(path, dest);
+    }
+
+    private long fileCount = 0;
+
+    @Override
+    protected Path getCacheFilePath() throws IOException {
+      fileCount++;
+      return Paths.get(Long.toString(fileCount));
+    }
+
+    @Override
+    public void close() throws IOException {
+      this.files.clear();
+    }
+  }
+
+  private static final Random RANDOM = new Random();
+
+  private static void randomDelay(int delay) {
+    try {
+      Thread.sleep(RANDOM.nextInt(delay));
+    } catch (InterruptedException e) {
+
+    }
+  }
+
+  public static class FakeS3ACachingBlockManager
+      extends S3ACachingBlockManager {
+
+    public FakeS3ACachingBlockManager(
+        ExecutorServiceFuturePool futurePool,
+        S3ARemoteObjectReader reader,
+        BlockData blockData,
+        int bufferPoolSize) {
+      super(futurePool, reader, blockData, bufferPoolSize,
+          new EmptyS3AStatisticsContext().newInputStreamStatistics());
+    }
+
+    @Override
+    public int read(ByteBuffer buffer, long offset, int size)
+        throws IOException {
+      randomDelay(100);
+      return this.getReader().read(buffer, offset, size);
+    }
+
+    @Override
+    protected BlockCache createCache() {
+      final int readDelayMs = 50;
+      final int writeDelayMs = 200;
+      return new FakeS3FilePerBlockCache(readDelayMs, writeDelayMs);
+    }
+  }
+
+  public static class FakeS3ACachingInputStream extends S3ACachingInputStream {
+
+    public FakeS3ACachingInputStream(
+        S3AReadOpContext context,
+        S3ObjectAttributes s3Attributes,
+        S3AInputStream.InputStreamCallbacks client,
+        S3AInputStreamStatistics streamStatistics) {
+      super(context, s3Attributes, client, streamStatistics);
+    }
+
+    @Override
+    protected S3ARemoteObject getS3File() {
+      randomDelay(200);
+      return new MockS3ARemoteObject(
+          (int) this.getS3ObjectAttributes().getLen(), false);
+    }
+
+    @Override
+    protected S3ACachingBlockManager createBlockManager(
+        ExecutorServiceFuturePool futurePool,
+        S3ARemoteObjectReader reader,
+        BlockData blockData,
+        int bufferPoolSize) {
+      return new FakeS3ACachingBlockManager(futurePool, reader, blockData,
+          bufferPoolSize);
+    }
+  }
+}

+ 87 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java

@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+
+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 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);
+    MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
+    S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
+
+    // Should not throw.
+    new S3ABlockManager(reader, blockData);
+
+    // Verify it throws correctly.
+    intercept(
+        IllegalArgumentException.class,
+        "'reader' must not be null",
+        () -> new S3ABlockManager(null, blockData));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'blockData' must not be null",
+        () -> new S3ABlockManager(reader, null));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> new S3ABlockManager(reader, blockData).get(-1));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'data' must not be null",
+        () -> new S3ABlockManager(reader, blockData).release(null));
+  }
+
+  @Test
+  public void testGet() throws IOException {
+    BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE);
+    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);
+      ByteBuffer buffer = data.getBuffer();
+      long startOffset = blockData.getStartOffset(b);
+      for (int i = 0; i < BLOCK_SIZE; i++) {
+        assertEquals(startOffset + i, buffer.get());
+      }
+    }
+  }
+}

+ 361 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java

@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.junit.Test;
+
+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 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 S3AInputStreamStatistics streamStatistics =
+      new EmptyS3AStatisticsContext().newInputStreamStatistics();
+
+  private final BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE);
+
+  @Test
+  public void testArgChecks() throws Exception {
+    MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
+    S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
+
+    // Should not throw.
+    S3ACachingBlockManager blockManager =
+        new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
+            streamStatistics);
+
+    // Verify it throws correctly.
+    intercept(
+        NullPointerException.class,
+        () -> new S3ACachingBlockManager(null, reader, blockData, POOL_SIZE,
+            streamStatistics));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'reader' must not be null",
+        () -> new S3ACachingBlockManager(futurePool, null, blockData, POOL_SIZE,
+            streamStatistics));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'blockData' must not be null",
+        () -> new S3ACachingBlockManager(futurePool, reader, null, POOL_SIZE,
+            streamStatistics));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'bufferPoolSize' must be a positive integer",
+        () -> new S3ACachingBlockManager(futurePool, reader, blockData, 0,
+            streamStatistics));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'bufferPoolSize' must be a positive integer",
+        () -> new S3ACachingBlockManager(futurePool, reader, blockData, -1,
+            streamStatistics));
+
+    intercept(NullPointerException.class,
+        () -> new S3ACachingBlockManager(futurePool, reader, blockData,
+            POOL_SIZE, null));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> blockManager.get(-1));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'data' must not be null",
+        () -> blockManager.release(null));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'blockNumber' must not be negative",
+        () -> blockManager.requestPrefetch(-1));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'data' must not be null",
+        () -> blockManager.requestCaching(null));
+  }
+
+  /**
+   * Extends S3ACachingBlockManager so that we can inject asynchronous failures.
+   */
+  private static final class BlockManagerForTesting
+      extends S3ACachingBlockManager {
+
+    BlockManagerForTesting(
+        ExecutorServiceFuturePool futurePool,
+        S3ARemoteObjectReader reader,
+        BlockData blockData,
+        int bufferPoolSize,
+        S3AInputStreamStatistics streamStatistics) {
+      super(futurePool, reader, blockData, bufferPoolSize, streamStatistics);
+    }
+
+    // If true, forces the next read operation to fail.
+    // Resets itself to false after one failure.
+    private boolean forceNextReadToFail;
+
+    @Override
+    public int read(ByteBuffer buffer, long offset, int size)
+        throws IOException {
+      if (forceNextReadToFail) {
+        forceNextReadToFail = false;
+        throw new RuntimeException("foo");
+      } else {
+        return super.read(buffer, offset, size);
+      }
+    }
+
+    // If true, forces the next cache-put operation to fail.
+    // Resets itself to false after one failure.
+    private boolean forceNextCachePutToFail;
+
+    @Override
+    protected void cachePut(int blockNumber, ByteBuffer buffer)
+        throws IOException {
+      if (forceNextCachePutToFail) {
+        forceNextCachePutToFail = false;
+        throw new RuntimeException("bar");
+      } else {
+        super.cachePut(blockNumber, buffer);
+      }
+    }
+  }
+
+  // @Ignore
+  @Test
+  public void testGet() throws Exception {
+    testGetHelper(false);
+  }
+
+  // @Ignore
+  @Test
+  public void testGetFailure() throws Exception {
+    testGetHelper(true);
+  }
+
+  private void testGetHelper(boolean forceReadFailure) throws Exception {
+    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.
+      boolean forceFailure = forceReadFailure && (b % 2 == 0);
+
+      BufferData data = null;
+
+      if (forceFailure) {
+        blockManager.forceNextReadToFail = true;
+
+        intercept(
+            RuntimeException.class,
+            "foo",
+            () -> blockManager.get(3));
+      } else {
+        data = blockManager.get(b);
+
+        long startOffset = blockData.getStartOffset(b);
+        for (int i = 0; i < blockData.getSize(b); i++) {
+          assertEquals(startOffset + i, data.getBuffer().get());
+        }
+
+        blockManager.release(data);
+      }
+
+      assertEquals(POOL_SIZE, blockManager.numAvailable());
+    }
+  }
+
+  // @Ignore
+  @Test
+  public void testPrefetch() throws IOException, InterruptedException {
+    testPrefetchHelper(false);
+  }
+
+  // @Ignore
+  @Test
+  public void testPrefetchFailure() throws IOException, InterruptedException {
+    testPrefetchHelper(true);
+  }
+
+  private void testPrefetchHelper(boolean forcePrefetchFailure)
+      throws IOException, InterruptedException {
+    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;
+    int expectedNumSuccesses = 0;
+
+    for (int b = 0; b < POOL_SIZE; b++) {
+      // We simulate caching failure for all odd numbered blocks.
+      boolean forceFailure = forcePrefetchFailure && (b % 2 == 1);
+      if (forceFailure) {
+        expectedNumErrors++;
+        blockManager.forceNextReadToFail = true;
+      } else {
+        expectedNumSuccesses++;
+      }
+      blockManager.requestPrefetch(b);
+    }
+
+    assertEquals(0, blockManager.numCached());
+
+    blockManager.cancelPrefetches();
+    waitForCaching(blockManager, expectedNumSuccesses);
+    assertEquals(expectedNumErrors, this.totalErrors(blockManager));
+    assertEquals(expectedNumSuccesses, blockManager.numCached());
+  }
+
+  // @Ignore
+  @Test
+  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++) {
+      blockManager.requestPrefetch(b);
+      BufferData data = blockManager.get(b);
+      blockManager.requestCaching(data);
+    }
+
+    waitForCaching(blockManager, blockData.getNumBlocks());
+    assertEquals(blockData.getNumBlocks(), blockManager.numCached());
+    assertEquals(0, this.totalErrors(blockManager));
+  }
+
+  // @Ignore
+  @Test
+  public void testCachingOfGet() throws IOException, InterruptedException {
+    testCachingOfGetHelper(false);
+  }
+
+  // @Ignore
+  @Test
+  public void testCachingFailureOfGet()
+      throws IOException, InterruptedException {
+    testCachingOfGetHelper(true);
+  }
+
+  public void testCachingOfGetHelper(boolean forceCachingFailure)
+      throws IOException, InterruptedException {
+    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;
+    int expectedNumSuccesses = 0;
+
+    for (int b = 0; b < blockData.getNumBlocks(); b++) {
+      // We simulate caching failure for all odd numbered blocks.
+      boolean forceFailure = forceCachingFailure && (b % 2 == 1);
+      if (forceFailure) {
+        expectedNumErrors++;
+      } else {
+        expectedNumSuccesses++;
+      }
+
+      BufferData data = blockManager.get(b);
+      if (forceFailure) {
+        blockManager.forceNextCachePutToFail = true;
+      }
+
+      blockManager.requestCaching(data);
+      waitForCaching(blockManager, expectedNumSuccesses);
+      assertEquals(expectedNumSuccesses, blockManager.numCached());
+
+      if (forceCachingFailure) {
+        assertEquals(expectedNumErrors, this.totalErrors(blockManager));
+      } else {
+        assertEquals(0, this.totalErrors(blockManager));
+      }
+    }
+  }
+
+  private void waitForCaching(
+      S3ACachingBlockManager blockManager,
+      int expectedCount)
+      throws InterruptedException {
+    // Wait for async cache operation to be over.
+    int numTrys = 0;
+    int count;
+    do {
+      Thread.sleep(100);
+      count = blockManager.numCached();
+      numTrys++;
+      if (numTrys > 600) {
+        String message = String.format(
+            "waitForCaching: expected: %d, actual: %d, read errors: %d, caching errors: %d",
+            expectedCount, count, blockManager.numReadErrors(),
+            blockManager.numCachingErrors());
+        throw new IllegalStateException(message);
+      }
+    }
+    while (count < expectedCount);
+  }
+
+  private int totalErrors(S3ACachingBlockManager blockManager) {
+    return blockManager.numCachingErrors() + blockManager.numReadErrors();
+  }
+
+  private void assertInitialState(S3ACachingBlockManager blockManager) {
+    assertEquals(0, blockManager.numCached());
+  }
+}

+ 279 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java

@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+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;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Applies the same set of tests to both S3ACachingInputStream and S3AInMemoryInputStream.
+ */
+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 =
+      MockS3ARemoteObject.createClient("bucket");
+
+  @Test
+  public void testArgChecks() throws Exception {
+    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 S3ACachingInputStream(readContext, attrs, client, stats);
+
+    ExceptionAsserts.assertThrows(
+        NullPointerException.class,
+        () -> new S3ACachingInputStream(null, attrs, client, stats));
+
+    ExceptionAsserts.assertThrows(
+        NullPointerException.class,
+        () -> new S3ACachingInputStream(readContext, null, client, stats));
+
+    ExceptionAsserts.assertThrows(
+        NullPointerException.class,
+        () -> new S3ACachingInputStream(readContext, attrs, null, stats));
+
+    ExceptionAsserts.assertThrows(
+        NullPointerException.class,
+        () -> new S3ACachingInputStream(readContext, attrs, client, null));
+  }
+
+  @Test
+  public void testRead0SizedFile() throws Exception {
+    S3ARemoteInputStream inputStream =
+        S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
+            "key", 0);
+    testRead0SizedFileHelper(inputStream, 9);
+
+    inputStream =
+        S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
+            0, 5, 2);
+    testRead0SizedFileHelper(inputStream, 5);
+  }
+
+  private void testRead0SizedFileHelper(S3ARemoteInputStream inputStream,
+      int bufferSize)
+      throws Exception {
+    assertEquals(0, inputStream.available());
+    assertEquals(-1, inputStream.read());
+    assertEquals(-1, inputStream.read());
+
+    byte[] buffer = new byte[2];
+    assertEquals(-1, inputStream.read(buffer));
+    assertEquals(-1, inputStream.read());
+  }
+
+  @Test
+  public void testRead() throws Exception {
+    S3ARemoteInputStream inputStream =
+        S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
+            "key", FILE_SIZE);
+    testReadHelper(inputStream, FILE_SIZE);
+
+    inputStream =
+        S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
+            FILE_SIZE, 5, 2);
+    testReadHelper(inputStream, 5);
+  }
+
+  private void testReadHelper(S3ARemoteInputStream inputStream, int bufferSize)
+      throws Exception {
+    assertEquals(bufferSize, inputStream.available());
+    assertEquals(0, inputStream.read());
+    assertEquals(1, inputStream.read());
+
+    byte[] buffer = new byte[2];
+    assertEquals(2, inputStream.read(buffer));
+    assertEquals(2, buffer[0]);
+    assertEquals(3, buffer[1]);
+
+    assertEquals(4, inputStream.read());
+
+    buffer = new byte[10];
+    int curPos = (int) inputStream.getPos();
+    int expectedRemainingBytes = (int) (FILE_SIZE - curPos);
+    int readStartOffset = 2;
+    assertEquals(
+        expectedRemainingBytes,
+        inputStream.read(buffer, readStartOffset, expectedRemainingBytes));
+
+    for (int i = 0; i < expectedRemainingBytes; i++) {
+      assertEquals(curPos + i, buffer[readStartOffset + i]);
+    }
+
+    assertEquals(-1, inputStream.read());
+    Thread.sleep(100);
+    assertEquals(-1, inputStream.read());
+    assertEquals(-1, inputStream.read());
+    assertEquals(-1, inputStream.read(buffer));
+    assertEquals(-1, inputStream.read(buffer, 1, 3));
+  }
+
+  @Test
+  public void testSeek() throws Exception {
+    S3ARemoteInputStream inputStream;
+    inputStream =
+        S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
+            "key", 9);
+    testSeekHelper(inputStream, 9, 9);
+
+    inputStream =
+        S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
+            9, 5, 1);
+    testSeekHelper(inputStream, 5, 9);
+  }
+
+  private void testSeekHelper(S3ARemoteInputStream inputStream,
+      int bufferSize,
+      int fileSize)
+      throws Exception {
+    assertEquals(0, inputStream.getPos());
+    inputStream.seek(7);
+    assertEquals(7, inputStream.getPos());
+    inputStream.seek(0);
+
+    assertEquals(bufferSize, inputStream.available());
+    for (int i = 0; i < fileSize; i++) {
+      assertEquals(i, inputStream.read());
+    }
+
+    for (int i = 0; i < fileSize; i++) {
+      inputStream.seek(i);
+      for (int j = i; j < fileSize; j++) {
+        assertEquals(j, inputStream.read());
+      }
+    }
+
+    // Test invalid seeks.
+    ExceptionAsserts.assertThrows(
+        EOFException.class,
+        FSExceptionMessages.NEGATIVE_SEEK,
+        () -> inputStream.seek(-1));
+  }
+
+  @Test
+  public void testRandomSeek() throws Exception {
+    S3ARemoteInputStream inputStream;
+    inputStream =
+        S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
+            "key", 9);
+    testRandomSeekHelper(inputStream, 9, 9);
+
+    inputStream =
+        S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
+            9, 5, 1);
+    testRandomSeekHelper(inputStream, 5, 9);
+  }
+
+  private void testRandomSeekHelper(S3ARemoteInputStream inputStream,
+      int bufferSize,
+      int fileSize)
+      throws Exception {
+    assertEquals(0, inputStream.getPos());
+    inputStream.seek(7);
+    assertEquals(7, inputStream.getPos());
+    inputStream.seek(0);
+
+    assertEquals(bufferSize, inputStream.available());
+    for (int i = 0; i < fileSize; i++) {
+      assertEquals(i, inputStream.read());
+    }
+
+    for (int i = 0; i < fileSize; i++) {
+      inputStream.seek(i);
+      for (int j = i; j < fileSize; j++) {
+        assertEquals(j, inputStream.read());
+      }
+
+      int seekFromEndPos = fileSize - i - 1;
+      inputStream.seek(seekFromEndPos);
+      for (int j = seekFromEndPos; j < fileSize; j++) {
+        assertEquals(j, inputStream.read());
+      }
+    }
+  }
+
+  @Test
+  public void testClose() throws Exception {
+    S3ARemoteInputStream inputStream =
+        S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
+            "key", 9);
+    testCloseHelper(inputStream, 9);
+
+    inputStream =
+        S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
+            9, 5, 3);
+    testCloseHelper(inputStream, 5);
+  }
+
+  private void testCloseHelper(S3ARemoteInputStream inputStream, int bufferSize)
+      throws Exception {
+    assertEquals(bufferSize, inputStream.available());
+    assertEquals(0, inputStream.read());
+    assertEquals(1, inputStream.read());
+
+    inputStream.close();
+
+    ExceptionAsserts.assertThrows(
+        IOException.class,
+        FSExceptionMessages.STREAM_IS_CLOSED,
+        () -> inputStream.available());
+
+    ExceptionAsserts.assertThrows(
+        IOException.class,
+        FSExceptionMessages.STREAM_IS_CLOSED,
+        () -> inputStream.read());
+
+    byte[] buffer = new byte[10];
+    ExceptionAsserts.assertThrows(
+        IOException.class,
+        FSExceptionMessages.STREAM_IS_CLOSED,
+        () -> inputStream.read(buffer));
+
+    // Verify a second close() does not throw.
+    inputStream.close();
+  }
+}

+ 88 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java

@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.junit.Test;
+
+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;
+import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.test.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 =
+      MockS3ARemoteObject.createClient("bucket");
+
+  @Test
+  public void testArgChecks() throws Exception {
+    S3AReadOpContext readContext =
+        S3APrefetchFakes.createReadContext(futurePool, "key", 10, 10, 1);
+    S3ObjectAttributes attrs =
+        S3APrefetchFakes.createObjectAttributes("bucket", "key", 10);
+    S3AInputStreamStatistics stats =
+        readContext.getS3AStatisticsContext().newInputStreamStatistics();
+    ChangeTracker changeTracker =
+        S3APrefetchFakes.createChangeTracker("bucket", "key", 10);
+
+    // Should not throw.
+    new S3ARemoteObject(readContext, attrs, client, stats, changeTracker);
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'context' must not be null",
+        () -> new S3ARemoteObject(null, attrs, client, stats, changeTracker));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'s3Attributes' must not be null",
+        () -> new S3ARemoteObject(readContext, null, client, stats,
+            changeTracker));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'client' must not be null",
+        () -> new S3ARemoteObject(readContext, attrs, null, stats,
+            changeTracker));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'streamStatistics' must not be null",
+        () -> new S3ARemoteObject(readContext, attrs, client, null,
+            changeTracker));
+
+    ExceptionAsserts.assertThrows(
+        IllegalArgumentException.class,
+        "'changeTracker' must not be null",
+        () -> new S3ARemoteObject(readContext, attrs, client, stats, null));
+  }
+}

+ 112 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java

@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.prefetch;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.junit.Assert.assertEquals;
+
+public class TestS3ARemoteObjectReader extends AbstractHadoopTestBase {
+
+  private static final int FILE_SIZE = 9;
+
+  private static final int BUFFER_SIZE = 2;
+
+  private final S3ARemoteObject remoteObject =
+      new MockS3ARemoteObject(FILE_SIZE, false);
+
+  @Test
+  public void testArgChecks() throws Exception {
+    // Should not throw.
+    S3ARemoteObjectReader reader = new S3ARemoteObjectReader(remoteObject);
+
+    // Verify it throws correctly.
+
+    intercept(
+        IllegalArgumentException.class,
+        "'remoteObject' must not be null",
+        () -> new S3ARemoteObjectReader(null));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'buffer' must not be null",
+        () -> reader.read(null, 10, 2));
+
+    ByteBuffer buffer = ByteBuffer.allocate(BUFFER_SIZE);
+
+    intercept(
+        IllegalArgumentException.class,
+        "'offset' (-1) must be within the range [0, 9]",
+        () -> reader.read(buffer, -1, 2));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'offset' (11) must be within the range [0, 9]",
+        () -> reader.read(buffer, 11, 2));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'size' must be a positive integer",
+        () -> reader.read(buffer, 1, 0));
+
+    intercept(
+        IllegalArgumentException.class,
+        "'size' must be a positive integer",
+        () -> reader.read(buffer, 1, -1));
+  }
+
+  @Test
+  public void testGetWithOffset() throws Exception {
+    for (int i = 0; i < FILE_SIZE; i++) {
+      testGetHelper(false, i);  // no retry
+      testGetHelper(true, i);   // with retry
+    }
+  }
+
+  private void testGetHelper(boolean testWithRetry, long startOffset)
+      throws Exception {
+    int numBlocks = 0;
+    ByteBuffer buffer;
+    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);
+      for (int readSize = 1; readSize <= FILE_SIZE; readSize++) {
+        buffer.clear();
+        int numBytesRead = reader.read(buffer, startOffset, readSize);
+        int expectedNumBytesRead = Math.min(readSize, remainingSize);
+        expectedNumBytesRead = Math.min(bufferSize, expectedNumBytesRead);
+        assertEquals(expectedNumBytesRead, numBytesRead);
+
+        byte[] bytes = buffer.array();
+        for (int i = 0; i < expectedNumBytesRead; i++) {
+          assertEquals(startOffset + i, bytes[i]);
+        }
+      }
+    }
+  }
+}

+ 9 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AInputPolicy;
 import org.apache.hadoop.fs.s3a.S3AInputStream;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
 import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
@@ -94,6 +95,14 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
   private boolean testDataAvailable = true;
   private String assumptionMessage = "test file";
 
+  @Override
+  protected Configuration createScaleConfiguration() {
+    Configuration conf = super.createScaleConfiguration();
+    S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
+    conf.setBoolean(PREFETCH_ENABLED_KEY, false);
+    return conf;
+  }
+
   /**
    * Open the FS and the test data. The input stream is always set up here.
    * @throws IOException IO Problems.