You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by mt...@apache.org on 2022/06/27 16:38:47 UTC

[hadoop] branch branch-3.3 updated (4ba463069bd -> c517b086f2c)

This is an automated email from the ASF dual-hosted git repository.

mthakur pushed a change to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git


    from 4ba463069bd HADOOP-18288. Total requests and total requests per sec served by RPC servers (#4485)
     new 5c348c41ab8 HADOOP-11867. Add a high-performance vectored read API. (#3904)
     new 9f03f879633 HADOOP-18104: S3A: Add configs to configure minSeekForVectorReads and maxReadSizeForVectorReads (#3964)
     new bb5a17b1771 HADOOP-18107 Adding scale test for vectored reads for large file (#4273)
     new bfb7d020d10 HADOOP-18105 Implement buffer pooling with weak references (#4263)
     new c517b086f2c HADOOP-18106: Handle memory fragmentation in S3A Vectored IO. (#4445)

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 dev-support/Jenkinsfile                            |   2 +-
 .../apache/hadoop/fs/BufferedFSInputStream.java    |  27 +-
 .../org/apache/hadoop/fs/ChecksumFileSystem.java   | 213 +++++++++--
 .../org/apache/hadoop/fs/FSDataInputStream.java    |  22 +-
 .../main/java/org/apache/hadoop/fs/FileRange.java  |  67 ++++
 .../org/apache/hadoop/fs/PositionedReadable.java   |  41 ++-
 .../org/apache/hadoop/fs/RawLocalFileSystem.java   | 110 +++++-
 .../org/apache/hadoop/fs/StreamCapabilities.java   |   6 +
 .../org/apache/hadoop/fs/VectoredReadUtils.java    | 292 +++++++++++++++
 .../apache/hadoop/fs/impl/CombinedFileRange.java   |  70 ++++
 .../org/apache/hadoop/fs/impl/FileRangeImpl.java   |  74 ++++
 .../java/org/apache/hadoop/io/ByteBufferPool.java  |   5 +
 .../apache/hadoop/io/ElasticByteBufferPool.java    |   4 +-
 .../io/WeakReferencedElasticByteBufferPool.java    | 155 ++++++++
 .../site/markdown/filesystem/fsdatainputstream.md  |  39 ++
 .../apache/hadoop/fs/TestVectoredReadUtils.java    | 371 +++++++++++++++++++
 .../contract/AbstractContractVectoredReadTest.java | 406 +++++++++++++++++++++
 .../hadoop/fs/contract/ContractTestUtils.java      |  84 +++++
 .../localfs/TestLocalFSContractVectoredRead.java   |  86 +++++
 .../rawlocal/TestRawLocalContractVectoredRead.java |  35 ++
 ...estMoreWeakReferencedElasticByteBufferPool.java |  97 +++++
 .../TestWeakReferencedElasticByteBufferPool.java   | 232 ++++++++++++
 .../java/org/apache/hadoop/test/MoreAsserts.java   |  49 ++-
 hadoop-common-project/pom.xml                      |   1 -
 hadoop-project/pom.xml                             |  11 +
 .../java/org/apache/hadoop/fs/s3a/Constants.java   |  26 ++
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java    |  39 +-
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   | 391 +++++++++++++++++++-
 .../org/apache/hadoop/fs/s3a/S3AReadOpContext.java |  20 +-
 .../apache/hadoop/fs/s3a/VectoredIOContext.java    |  78 ++++
 .../fs/s3a/impl/GetContentSummaryOperation.java    |   3 +-
 .../site/markdown/tools/hadoop-aws/performance.md  |  30 ++
 .../contract/s3a/ITestS3AContractVectoredRead.java | 159 ++++++++
 .../hadoop/fs/s3a/TestS3AInputStreamRetry.java     |   3 +-
 .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java    |  32 ++
 .../hadoop-aws/src/test/resources/log4j.properties |   2 +-
 hadoop-tools/hadoop-benchmark/pom.xml              |  94 +++++
 .../hadoop-benchmark/src/main/assembly/uber.xml    |  33 ++
 .../hadoop-benchmark/src/main/findbugs/exclude.xml |  22 ++
 .../hadoop/benchmark/VectoredReadBenchmark.java    | 245 +++++++++++++
 .../org/apache/hadoop/benchmark/package-info.java  |  22 ++
 hadoop-tools/pom.xml                               |   1 +
 42 files changed, 3602 insertions(+), 97 deletions(-)
 create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java
 create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java
 create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java
 create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileRangeImpl.java
 create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java
 create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java
 create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
 create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java
 create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractVectoredRead.java
 create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMoreWeakReferencedElasticByteBufferPool.java
 create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWeakReferencedElasticByteBufferPool.java
 create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java
 create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java
 create mode 100644 hadoop-tools/hadoop-benchmark/pom.xml
 create mode 100644 hadoop-tools/hadoop-benchmark/src/main/assembly/uber.xml
 create mode 100644 hadoop-tools/hadoop-benchmark/src/main/findbugs/exclude.xml
 create mode 100644 hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java
 create mode 100644 hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/package-info.java


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[hadoop] 02/05: HADOOP-18104: S3A: Add configs to configure minSeekForVectorReads and maxReadSizeForVectorReads (#3964)

Posted by mt...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mthakur pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 9f03f879633c201e04e594c43e6694d2fc64f0bb
Author: Mukund Thakur <mt...@cloudera.com>
AuthorDate: Sat Apr 30 04:17:33 2022 +0530

    HADOOP-18104: S3A: Add configs to configure minSeekForVectorReads and maxReadSizeForVectorReads (#3964)
    
    Part of HADOOP-18103.
    Introducing fs.s3a.vectored.read.min.seek.size and fs.s3a.vectored.read.max.merged.size
    to configure min seek and max read during a vectored IO operation in S3A connector.
    These properties actually define how the ranges will be merged. To completely
    disable merging set fs.s3a.max.readsize.vectored.read to 0.
    
    Contributed By: Mukund Thakur
---
 .../site/markdown/filesystem/fsdatainputstream.md  |  1 +
 .../contract/AbstractContractVectoredReadTest.java | 21 +++---
 .../hadoop/fs/impl/TestVectoredReadUtils.java      | 24 +++++++
 .../java/org/apache/hadoop/test/MoreAsserts.java   | 12 ++++
 .../java/org/apache/hadoop/fs/s3a/Constants.java   | 26 ++++++++
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java    | 25 ++++++-
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   | 21 ++++++
 .../org/apache/hadoop/fs/s3a/S3AReadOpContext.java | 21 +++++-
 .../apache/hadoop/fs/s3a/VectoredIOContext.java    | 78 ++++++++++++++++++++++
 .../site/markdown/tools/hadoop-aws/performance.md  | 30 +++++++++
 .../contract/s3a/ITestS3AContractVectoredRead.java | 54 ++++++++++++++-
 11 files changed, 297 insertions(+), 16 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
index 0fe1772d266..e4a2830967e 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
@@ -474,6 +474,7 @@ end of first and start of next range is more than this value.
 
 Maximum number of bytes which can be read in one go after merging the ranges.
 Two ranges won't be merged if the combined data to be read is more than this value.
+Essentially setting this to 0 will disable the merging of ranges.
 
 ## Consistency
 
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
index eee4b11e739..756c3de85cc 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
@@ -18,15 +18,6 @@
 
 package org.apache.hadoop.fs.contract;
 
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileRange;
-import org.apache.hadoop.fs.FileRangeImpl;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.impl.FutureIOSupport;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -42,6 +33,15 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.FileRangeImpl;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.impl.FutureIOSupport;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 
@@ -52,7 +52,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
 
   public static final int DATASET_LEN = 64 * 1024;
   private static final byte[] DATASET = ContractTestUtils.dataset(DATASET_LEN, 'a', 32);
-  private static final String VECTORED_READ_FILE_NAME = "vectored_file.txt";
+  protected static final String VECTORED_READ_FILE_NAME = "vectored_file.txt";
   private static final String VECTORED_READ_FILE_1MB_NAME = "vectored_file_1M.txt";
   private static final byte[] DATASET_MB = ContractTestUtils.dataset(1024 * 1024, 'a', 256);
 
@@ -172,6 +172,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
     }
   }
 
+  @Test
   public void testSameRanges() throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java
index f789f361905..cfd366701be 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java
@@ -207,6 +207,30 @@ public class TestVectoredReadUtils extends HadoopTestBase {
             VectoredReadUtils.isOrderedDisjoint(outputList, 1, 800));
 
   }
+
+  @Test
+  public void testMaxSizeZeroDisablesMering() throws Exception {
+    List<FileRange> randomRanges = Arrays.asList(
+            new FileRangeImpl(3000, 110),
+            new FileRangeImpl(3000, 100),
+            new FileRangeImpl(2100, 100)
+    );
+    assertEqualRangeCountsAfterMerging(randomRanges, 1, 1, 0);
+    assertEqualRangeCountsAfterMerging(randomRanges, 1, 0, 0);
+    assertEqualRangeCountsAfterMerging(randomRanges, 1, 100, 0);
+  }
+
+  private void assertEqualRangeCountsAfterMerging(List<FileRange> inputRanges,
+                                                  int chunkSize,
+                                                  int minimumSeek,
+                                                  int maxSize) {
+    List<CombinedFileRange> combinedFileRanges = VectoredReadUtils
+            .sortAndMergeRanges(inputRanges, chunkSize, minimumSeek, maxSize);
+    Assertions.assertThat(combinedFileRanges)
+            .describedAs("Mismatch in number of ranges post merging")
+            .hasSize(inputRanges.size());
+  }
+
   interface Stream extends PositionedReadable, ByteBufferPositionedReadable {
     // nothing
   }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java
index f83ef9e63d1..f6e6055d78e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java
@@ -86,4 +86,16 @@ public class MoreAsserts {
                     "completed exceptionally")
             .isTrue();
   }
+
+  /**
+   * Assert two same type of values.
+   * @param actual actual value.
+   * @param expected expected value.
+   * @param message error message to print in case of mismatch.
+   */
+  public static <T> void assertEqual(T actual, T expected, String message) {
+    Assertions.assertThat(actual)
+            .describedAs("Mismatch in %s", message)
+            .isEqualTo(expected);
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index 48d17aa402d..764a6adaca2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -1177,4 +1177,30 @@ public final class Constants {
    */
   public static final String FS_S3A_CREATE_HEADER = "fs.s3a.create.header";
 
+  /**
+   * What is the smallest reasonable seek in bytes such
+   * that we group ranges together during vectored read operation.
+   * Value : {@value}.
+   */
+  public static final String AWS_S3_VECTOR_READS_MIN_SEEK_SIZE =
+          "fs.s3a.vectored.read.min.seek.size";
+
+  /**
+   * What is the largest merged read size in bytes such
+   * that we group ranges together during vectored read.
+   * Setting this value to 0 will disable merging of ranges.
+   * Value : {@value}.
+   */
+  public static final String AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE =
+          "fs.s3a.vectored.read.max.merged.size";
+
+  /**
+   * Default minimum seek in bytes during vectored reads : {@value}.
+   */
+  public static final int DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE = 4896; // 4K
+
+  /**
+   * Default maximum read size in bytes during vectored reads : {@value}.
+   */
+  public static final int DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE = 1253376; //1M
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 943416e2d87..ac2099bedce 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -313,6 +313,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * {@code openFile()}.
    */
   private S3AInputPolicy inputPolicy;
+  /** Vectored IO context. */
+  private VectoredIOContext vectoredIOContext;
+
+  private long readAhead;
   private ChangeDetectionPolicy changeDetectionPolicy;
   private final AtomicBoolean closed = new AtomicBoolean(false);
   private volatile boolean isClosed = false;
@@ -584,6 +588,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           longBytesOption(conf, ASYNC_DRAIN_THRESHOLD,
                         DEFAULT_ASYNC_DRAIN_THRESHOLD, 0),
           inputPolicy);
+      vectoredIOContext = populateVectoredIOContext(conf);
     } catch (AmazonClientException e) {
       // amazon client exception: stop all services then throw the translation
       cleanupWithLogger(LOG, span);
@@ -597,6 +602,23 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     }
   }
 
+  /**
+   * Populates the configurations related to vectored IO operation
+   * in the context which has to passed down to input streams.
+   * @param conf configuration object.
+   * @return VectoredIOContext.
+   */
+  private VectoredIOContext populateVectoredIOContext(Configuration conf) {
+    final int minSeekVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MIN_SEEK_SIZE,
+            DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, 0);
+    final int maxReadSizeVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE,
+            DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, 0);
+    return new VectoredIOContext()
+            .setMinSeekForVectoredReads(minSeekVectored)
+            .setMaxReadSizeForVectoredReads(maxReadSizeVectored)
+            .build();
+  }
+
   /**
    * Set the client side encryption gauge to 0 or 1, indicating if CSE is
    * enabled through the gauge or not.
@@ -1552,7 +1574,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         invoker,
         statistics,
         statisticsContext,
-        fileStatus)
+        fileStatus,
+        vectoredIOContext)
         .withAuditSpan(auditSpan);
     openFileHelper.applyDefaultOptions(roc);
     return roc.build();
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index 05d9c7f9fe9..23f31df1645 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -145,6 +145,9 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   private S3AInputPolicy inputPolicy;
   private long readahead = Constants.DEFAULT_READAHEAD_RANGE;
 
+  /** Vectored IO context. */
+  private final VectoredIOContext vectoredIOContext;
+
   /**
    * This is the actual position within the object, used by
    * lazy seek to decide whether to seek on the next read or not.
@@ -212,6 +215,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     setReadahead(ctx.getReadahead());
     this.asyncDrainThreshold = ctx.getAsyncDrainThreshold();
     this.unboundedThreadPool = unboundedThreadPool;
+    this.vectoredIOContext = context.getVectoredIOContext();
   }
 
   /**
@@ -859,6 +863,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       sb.append(" remainingInCurrentRequest=")
           .append(remainingInCurrentRequest());
       sb.append(" ").append(changeTracker);
+      sb.append(" ").append(vectoredIOContext);
       sb.append('\n').append(s);
       sb.append('}');
       return sb.toString();
@@ -905,6 +910,22 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     }
   }
 
+  /**
+   * {@inheritDoc}.
+   */
+  @Override
+  public int minSeekForVectorReads() {
+    return vectoredIOContext.getMinSeekForVectorReads();
+  }
+
+  /**
+   * {@inheritDoc}.
+   */
+  @Override
+  public int maxReadSizeForVectorReads() {
+    return vectoredIOContext.getMaxReadSizeForVectorReads();
+  }
+
   /**
    * {@inheritDoc}
    * Vectored read implementation for S3AInputStream.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
index 18ab14398a5..47de1d4d7d7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
@@ -64,6 +64,12 @@ public class S3AReadOpContext extends S3AOpContext {
    */
   private long asyncDrainThreshold;
 
+  /**
+   * Vectored IO context for vectored read api
+   * in {@code S3AInputStream#readVectored(List, IntFunction)}.
+   */
+  private final VectoredIOContext vectoredIOContext;
+
   /**
    * Instantiate.
    * @param path path of read
@@ -71,17 +77,19 @@ public class S3AReadOpContext extends S3AOpContext {
    * @param stats Fileystem statistics (may be null)
    * @param instrumentation statistics context
    * @param dstFileStatus target file status
+   * @param vectoredIOContext context for vectored read operation.
    */
   public S3AReadOpContext(
       final Path path,
       Invoker invoker,
       @Nullable FileSystem.Statistics stats,
       S3AStatisticsContext instrumentation,
-      FileStatus dstFileStatus) {
-
+      FileStatus dstFileStatus,
+      VectoredIOContext vectoredIOContext) {
     super(invoker, stats, instrumentation,
         dstFileStatus);
     this.path = requireNonNull(path);
+    this.vectoredIOContext = requireNonNull(vectoredIOContext, "vectoredIOContext");
   }
 
   /**
@@ -145,6 +153,7 @@ public class S3AReadOpContext extends S3AOpContext {
   }
 
   /**
+<<<<<<< HEAD
    * Set builder value.
    * @param value new value
    * @return the builder
@@ -199,6 +208,14 @@ public class S3AReadOpContext extends S3AOpContext {
     return asyncDrainThreshold;
   }
 
+  /**
+   * Get Vectored IO context for this this read op.
+   * @return vectored IO context.
+   */
+  public VectoredIOContext getVectoredIOContext() {
+    return vectoredIOContext;
+  }
+
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder(
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java
new file mode 100644
index 00000000000..31f0ae4cb55
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.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;
+
+import java.util.List;
+import java.util.function.IntFunction;
+
+/**
+ * Context related to vectored IO operation.
+ * See {@link S3AInputStream#readVectored(List, IntFunction)}.
+ */
+public class VectoredIOContext {
+
+  /**
+   * What is the smallest reasonable seek that we should group
+   * ranges together during vectored read operation.
+   */
+  private int minSeekForVectorReads;
+
+  /**
+   * What is the largest size that we should group ranges
+   * together during vectored read operation.
+   * Setting this value 0 will disable merging of ranges.
+   */
+  private int maxReadSizeForVectorReads;
+
+  /**
+   * Default no arg constructor.
+   */
+  public VectoredIOContext() {
+  }
+
+  public VectoredIOContext setMinSeekForVectoredReads(int minSeek) {
+    this.minSeekForVectorReads = minSeek;
+    return this;
+  }
+
+  public VectoredIOContext setMaxReadSizeForVectoredReads(int maxSize) {
+    this.maxReadSizeForVectorReads = maxSize;
+    return this;
+  }
+
+  public VectoredIOContext build() {
+    return this;
+  }
+
+  public int getMinSeekForVectorReads() {
+    return minSeekForVectorReads;
+  }
+
+  public int getMaxReadSizeForVectorReads() {
+    return maxReadSizeForVectorReads;
+  }
+
+  @Override
+  public String toString() {
+    return "VectoredIOContext{" +
+            "minSeekForVectorReads=" + minSeekForVectorReads +
+            ", maxReadSizeForVectorReads=" + maxReadSizeForVectorReads +
+            '}';
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md
index f398c4cbcbe..06eb137cd9b 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md
@@ -55,6 +55,36 @@ it isn't, and some attempts to preserve the metaphor are "aggressively suboptima
 
 To make most efficient use of S3, care is needed.
 
+## <a name="vectoredIO"></a> Improving read performance using Vectored IO
+The S3A FileSystem supports implementation of vectored read api using which
+a client can provide a list of file ranges to read returning a future read
+object associated with each range. For full api specification please see
+[FSDataInputStream](../../hadoop-common-project/hadoop-common/filesystem/fsdatainputstream.html).
+
+The following properties can be configured to optimise vectored reads based
+on the client requirements.
+
+```xml
+<property>
+  <name>fs.s3a.vectored.read.min.seek.size</name>
+  <value>4K</value>
+  <description>
+     What is the smallest reasonable seek in bytes such
+     that we group ranges together during vectored
+     read operation.
+   </description>
+</property>
+<property>
+<name>fs.s3a.vectored.read.max.merged.size</name>
+<value>1M</value>
+<description>
+   What is the largest merged read size in bytes such
+   that we group ranges together during vectored read.
+   Setting this value to 0 will disable merging of ranges.
+</description>
+</property>
+```
+
 ## <a name="fadvise"></a> Improving data input performance through fadvise
 
 The S3A Filesystem client supports the notion of input policies, similar
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java
index 255cc6501c2..0752e75d247 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java
@@ -18,15 +18,23 @@
 
 package org.apache.hadoop.fs.contract.s3a;
 
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileRange;
 import org.apache.hadoop.fs.FileRangeImpl;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
 
-import java.util.ArrayList;
-import java.util.List;
+import static org.apache.hadoop.test.MoreAsserts.assertEqual;
 
 public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTest {
 
@@ -42,7 +50,6 @@ public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTe
   /**
    * Overriding in S3 vectored read api fails fast in case of EOF
    * requested range.
-   * @throws Exception
    */
   @Override
   public void testEOFRanges() throws Exception {
@@ -51,4 +58,45 @@ public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTe
     fileRanges.add(new FileRangeImpl(DATASET_LEN, 100));
     testExceptionalVectoredRead(fs, fileRanges, "EOFException is expected");
   }
+
+  @Test
+  public void testMinSeekAndMaxSizeConfigsPropagation() throws Exception {
+    Configuration conf = getFileSystem().getConf();
+    S3ATestUtils.removeBaseAndBucketOverrides(conf,
+            Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE,
+            Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE);
+    S3ATestUtils.disableFilesystemCaching(conf);
+    final int configuredMinSeek = 2 * 1024;
+    final int configuredMaxSize = 10 * 1024 * 1024;
+    conf.set(Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, "2K");
+    conf.set(Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, "10M");
+    try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+      try (FSDataInputStream fis = fs.open(path(VECTORED_READ_FILE_NAME))) {
+        int newMinSeek = fis.minSeekForVectorReads();
+        int newMaxSize = fis.maxReadSizeForVectorReads();
+        assertEqual(newMinSeek, configuredMinSeek,
+                "configured s3a min seek for vectored reads");
+        assertEqual(newMaxSize, configuredMaxSize,
+                "configured s3a max size for vectored reads");
+      }
+    }
+  }
+
+  @Test
+  public void testMinSeekAndMaxSizeDefaultValues() throws Exception {
+    Configuration conf = getFileSystem().getConf();
+    S3ATestUtils.removeBaseAndBucketOverrides(conf,
+            Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE,
+            Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE);
+    try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+      try (FSDataInputStream fis = fs.open(path(VECTORED_READ_FILE_NAME))) {
+        int minSeek = fis.minSeekForVectorReads();
+        int maxSize = fis.maxReadSizeForVectorReads();
+        assertEqual(minSeek, Constants.DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE,
+                "default s3a min seek for vectored reads");
+        assertEqual(maxSize, Constants.DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE,
+                "default s3a max read size for vectored reads");
+      }
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[hadoop] 01/05: HADOOP-11867. Add a high-performance vectored read API. (#3904)

Posted by mt...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mthakur pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 5c348c41ab8ddb81146355570856e61e8d129a1e
Author: Mukund Thakur <mt...@cloudera.com>
AuthorDate: Tue Feb 1 19:52:38 2022 +0530

    HADOOP-11867. Add a high-performance vectored read API. (#3904)
    
    part of HADOOP-18103.
    Add support for multiple ranged vectored read api in PositionedReadable.
    The default iterates through the ranges to read each synchronously,
    but the intent is that FSDataInputStream subclasses can make more
    efficient readers especially in object stores implementation.
    
    Also added implementation in S3A where smaller ranges are merged and
    sliced byte buffers are returned to the readers. All the merged ranged are
    fetched from S3 asynchronously.
    
    Contributed By: Owen O'Malley and Mukund Thakur
    
     Conflicts:
            hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
            pom.xml
---
 dev-support/Jenkinsfile                            |   2 +-
 .../apache/hadoop/fs/BufferedFSInputStream.java    |  27 +-
 .../org/apache/hadoop/fs/ChecksumFileSystem.java   | 206 ++++++++---
 .../org/apache/hadoop/fs/FSDataInputStream.java    |  22 +-
 .../main/java/org/apache/hadoop/fs/FileRange.java  |  55 +++
 .../java/org/apache/hadoop/fs/FileRangeImpl.java   |  69 ++++
 .../org/apache/hadoop/fs/PositionedReadable.java   |  43 ++-
 .../org/apache/hadoop/fs/RawLocalFileSystem.java   | 108 +++++-
 .../apache/hadoop/fs/impl/CombinedFileRange.java   |  71 ++++
 .../apache/hadoop/fs/impl/VectoredReadUtils.java   | 277 +++++++++++++++
 .../site/markdown/filesystem/fsdatainputstream.md  |  31 ++
 .../contract/AbstractContractVectoredReadTest.java | 375 +++++++++++++++++++++
 .../localfs/TestLocalFSContractVectoredRead.java   |  35 ++
 .../rawlocal/TestRawLocalContractVectoredRead.java |  35 ++
 .../hadoop/fs/impl/TestVectoredReadUtils.java      | 344 +++++++++++++++++++
 .../java/org/apache/hadoop/test/MoreAsserts.java   |  37 +-
 hadoop-common-project/pom.xml                      |   1 -
 hadoop-project/pom.xml                             |  11 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java    |  14 +-
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   | 288 ++++++++++++++--
 .../contract/s3a/ITestS3AContractVectoredRead.java |  54 +++
 .../hadoop/fs/s3a/TestS3AInputStreamRetry.java     |   3 +-
 .../hadoop-aws/src/test/resources/log4j.properties |   2 +-
 hadoop-tools/hadoop-benchmark/pom.xml              |  94 ++++++
 .../hadoop-benchmark/src/main/assembly/uber.xml    |  33 ++
 .../hadoop-benchmark/src/main/findbugs/exclude.xml |  22 ++
 .../hadoop/benchmark/VectoredReadBenchmark.java    | 245 ++++++++++++++
 .../org/apache/hadoop/benchmark/package-info.java  |  22 ++
 hadoop-tools/pom.xml                               |   1 +
 29 files changed, 2438 insertions(+), 89 deletions(-)

diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 5b6fcb68f8d..9edd77b58e5 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -23,7 +23,7 @@ pipeline {
 
     options {
         buildDiscarder(logRotator(numToKeepStr: '5'))
-        timeout (time: 24, unit: 'HOURS')
+        timeout (time: 48, unit: 'HOURS')
         timestamps()
         checkoutToSubdirectory('src')
     }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java
index 59345f5d25c..7f3171235c8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -22,6 +22,9 @@ import java.io.EOFException;
 import java.io.FileDescriptor;
 import java.io.IOException;
 import java.util.StringJoiner;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.function.IntFunction;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -158,8 +161,24 @@ public class BufferedFSInputStream extends BufferedInputStream
   @Override
   public String toString() {
     return new StringJoiner(", ",
-        BufferedFSInputStream.class.getSimpleName() + "[", "]")
-        .add("in=" + in)
-        .toString();
+            BufferedFSInputStream.class.getSimpleName() + "[", "]")
+            .add("in=" + in)
+            .toString();
+  }
+
+  @Override
+  public int minSeekForVectorReads() {
+    return ((PositionedReadable) in).minSeekForVectorReads();
+  }
+
+  @Override
+  public int maxReadSizeForVectorReads() {
+    return ((PositionedReadable) in).maxReadSizeForVectorReads();
+  }
+
+  @Override
+  public void readVectored(List<? extends FileRange> ranges,
+                           IntFunction<ByteBuffer> allocate) throws IOException {
+    ((PositionedReadable) in).readVectored(ranges, allocate);
   }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
index 2ae5eabd49e..e612713cfed 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
@@ -22,17 +22,25 @@ import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.IntBuffer;
 import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.function.IntFunction;
+import java.util.zip.CRC32;
 
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
+import org.apache.hadoop.fs.impl.VectoredReadUtils;
+import org.apache.hadoop.fs.impl.CombinedFileRange;
 import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl;
 import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -66,7 +74,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   public static double getApproxChkSumLength(long size) {
     return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size;
   }
-  
+
   public ChecksumFileSystem(FileSystem fs) {
     super(fs);
   }
@@ -82,7 +90,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
           bytesPerChecksum);
     }
   }
-  
+
   /**
    * Set whether to verify checksum.
    */
@@ -95,7 +103,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   public void setWriteChecksum(boolean writeChecksum) {
     this.writeChecksum = writeChecksum;
   }
-  
+
   /** get the raw file system */
   @Override
   public FileSystem getRawFileSystem() {
@@ -162,18 +170,18 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     private ChecksumFileSystem fs;
     private FSDataInputStream datas;
     private FSDataInputStream sums;
-    
+
     private static final int HEADER_LENGTH = 8;
-    
+
     private int bytesPerSum = 1;
-    
+
     public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
       throws IOException {
       this(fs, file, fs.getConf().getInt(
-                       LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY, 
+                       LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY,
                        LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT));
     }
-    
+
     public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize)
       throws IOException {
       super( file, fs.getFileStatus(file).getReplication() );
@@ -189,7 +197,8 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
         if (!Arrays.equals(version, CHECKSUM_VERSION))
           throw new IOException("Not a checksum file: "+sumFile);
         this.bytesPerSum = sums.readInt();
-        set(fs.verifyChecksum, DataChecksum.newCrc32(), bytesPerSum, 4);
+        set(fs.verifyChecksum, DataChecksum.newCrc32(), bytesPerSum,
+            FSInputChecker.CHECKSUM_SIZE);
       } catch (IOException e) {
         // mincing the message is terrible, but java throws permission
         // exceptions as FNF because that's all the method signatures allow!
@@ -201,21 +210,21 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
         set(fs.verifyChecksum, null, 1, 0);
       }
     }
-    
+
     private long getChecksumFilePos( long dataPos ) {
-      return HEADER_LENGTH + 4*(dataPos/bytesPerSum);
+      return HEADER_LENGTH + FSInputChecker.CHECKSUM_SIZE*(dataPos/bytesPerSum);
     }
-    
+
     @Override
     protected long getChunkPosition( long dataPos ) {
       return dataPos/bytesPerSum*bytesPerSum;
     }
-    
+
     @Override
     public int available() throws IOException {
       return datas.available() + super.available();
     }
-    
+
     @Override
     public int read(long position, byte[] b, int off, int len)
       throws IOException {
@@ -233,7 +242,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       }
       return nread;
     }
-    
+
     @Override
     public void close() throws IOException {
       datas.close();
@@ -242,7 +251,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       }
       set(fs.verifyChecksum, null, 1, 0);
     }
-    
+
 
     @Override
     public boolean seekToNewSource(long targetPos) throws IOException {
@@ -265,7 +274,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
         final int checksumsToRead = Math.min(
           len/bytesPerSum, // number of checksums based on len to read
           checksum.length / CHECKSUM_SIZE); // size of checksum buffer
-        long checksumPos = getChecksumFilePos(pos); 
+        long checksumPos = getChecksumFilePos(pos);
         if(checksumPos != sums.getPos()) {
           sums.seek(checksumPos);
         }
@@ -305,8 +314,129 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     public IOStatistics getIOStatistics() {
       return IOStatisticsSupport.retrieveIOStatistics(datas);
     }
+
+    public static long findChecksumOffset(long dataOffset,
+                                          int bytesPerSum) {
+      return HEADER_LENGTH + (dataOffset/bytesPerSum) * FSInputChecker.CHECKSUM_SIZE;
+    }
+
+    /**
+     * Find the checksum ranges that correspond to the given data ranges.
+     * @param dataRanges the input data ranges, which are assumed to be sorted
+     *                   and non-overlapping
+     * @return a list of AsyncReaderUtils.CombinedFileRange that correspond to
+     *         the checksum ranges
+     */
+    public static List<CombinedFileRange> findChecksumRanges(
+        List<? extends FileRange> dataRanges,
+        int bytesPerSum,
+        int minSeek,
+        int maxSize) {
+      List<CombinedFileRange> result = new ArrayList<>();
+      CombinedFileRange currentCrc = null;
+      for(FileRange range: dataRanges) {
+        long crcOffset = findChecksumOffset(range.getOffset(), bytesPerSum);
+        long crcEnd = findChecksumOffset(range.getOffset() + range.getLength() +
+                                             bytesPerSum - 1, bytesPerSum);
+        if (currentCrc == null ||
+                !currentCrc.merge(crcOffset, crcEnd, range, minSeek, maxSize)) {
+          currentCrc = new CombinedFileRange(crcOffset, crcEnd, range);
+          result.add(currentCrc);
+        }
+      }
+      return result;
+    }
+
+    /**
+     * Check the data against the checksums.
+     * @param sumsBytes the checksum data
+     * @param sumsOffset where from the checksum file this buffer started
+     * @param data the file data
+     * @param dataOffset where the file data started (must be a multiple of
+     *                  bytesPerSum)
+     * @param bytesPerSum how many bytes per a checksum
+     * @param file the path of the filename
+     * @return the data buffer
+     * @throws CompletionException if the checksums don't match
+     */
+    static ByteBuffer checkBytes(ByteBuffer sumsBytes,
+                                 long sumsOffset,
+                                 ByteBuffer data,
+                                 long dataOffset,
+                                 int bytesPerSum,
+                                 Path file) {
+      // determine how many bytes we need to skip at the start of the sums
+      int offset =
+          (int) (findChecksumOffset(dataOffset, bytesPerSum) - sumsOffset);
+      IntBuffer sums = sumsBytes.asIntBuffer();
+      sums.position(offset / FSInputChecker.CHECKSUM_SIZE);
+      ByteBuffer current = data.duplicate();
+      int numChunks = data.remaining() / bytesPerSum;
+      CRC32 crc = new CRC32();
+      // check each chunk to ensure they match
+      for(int c = 0; c < numChunks; ++c) {
+        // set the buffer position and the limit
+        current.limit((c + 1) * bytesPerSum);
+        current.position(c * bytesPerSum);
+        // compute the crc
+        crc.reset();
+        crc.update(current);
+        int expected = sums.get();
+        int calculated = (int) crc.getValue();
+
+        if (calculated != expected) {
+          // cast of c added to silence findbugs
+          long errPosn = dataOffset + (long) c * bytesPerSum;
+          throw new CompletionException(new ChecksumException(
+              "Checksum error: " + file + " at " + errPosn +
+                  " exp: " + expected + " got: " + calculated, errPosn));
+        }
+      }
+      // if everything matches, we return the data
+      return data;
+    }
+
+    @Override
+    public void readVectored(List<? extends FileRange> ranges,
+                             IntFunction<ByteBuffer> allocate) throws IOException {
+      // If the stream doesn't have checksums, just delegate.
+      VectoredReadUtils.validateVectoredReadRanges(ranges);
+      if (sums == null) {
+        datas.readVectored(ranges, allocate);
+        return;
+      }
+      int minSeek = minSeekForVectorReads();
+      int maxSize = maxReadSizeForVectorReads();
+      List<CombinedFileRange> dataRanges =
+          VectoredReadUtils.sortAndMergeRanges(ranges, bytesPerSum,
+              minSeek, maxReadSizeForVectorReads());
+      List<CombinedFileRange> checksumRanges = findChecksumRanges(dataRanges,
+          bytesPerSum, minSeek, maxSize);
+      sums.readVectored(checksumRanges, allocate);
+      datas.readVectored(dataRanges, allocate);
+      // Data read is correct. I have verified content of dataRanges.
+      // There is some bug below here as test (testVectoredReadMultipleRanges)
+      // is failing, should be
+      // somewhere while slicing the merged data into smaller user ranges.
+      // Spend some time figuring out but it is a complex code.
+      for(CombinedFileRange checksumRange: checksumRanges) {
+        for(FileRange dataRange: checksumRange.getUnderlying()) {
+          // when we have both the ranges, validate the checksum
+          CompletableFuture<ByteBuffer> result =
+              checksumRange.getData().thenCombineAsync(dataRange.getData(),
+                  (sumBuffer, dataBuffer) ->
+                      checkBytes(sumBuffer, checksumRange.getOffset(),
+                          dataBuffer, dataRange.getOffset(), bytesPerSum, file));
+          // Now, slice the read data range to the user's ranges
+          for(FileRange original: ((CombinedFileRange) dataRange).getUnderlying()) {
+            original.setData(result.thenApply(
+                (b) -> VectoredReadUtils.sliceTo(b, dataRange.getOffset(), original)));
+          }
+        }
+      }
+    }
   }
-  
+
   private static class FSDataBoundedInputStream extends FSDataInputStream {
     private FileSystem fs;
     private Path file;
@@ -317,12 +447,12 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       this.fs = fs;
       this.file = file;
     }
-    
+
     @Override
     public boolean markSupported() {
       return false;
     }
-    
+
     /* Return the file length */
     private long getFileLength() throws IOException {
       if( fileLen==-1L ) {
@@ -330,7 +460,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       }
       return fileLen;
     }
-    
+
     /**
      * Skips over and discards <code>n</code> bytes of data from the
      * input stream.
@@ -354,11 +484,11 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       }
       return super.skip(n);
     }
-    
+
     /**
      * Seek to the given position in the stream.
      * The next read() will be from that position.
-     * 
+     *
      * <p>This method does not allow seek past the end of the file.
      * This produces IOException.
      *
@@ -424,22 +554,22 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
    */
   public static long getChecksumLength(long size, int bytesPerSum) {
     //the checksum length is equal to size passed divided by bytesPerSum +
-    //bytes written in the beginning of the checksum file.  
-    return ((size + bytesPerSum - 1) / bytesPerSum) * 4 +
-             CHECKSUM_VERSION.length + 4;  
+    //bytes written in the beginning of the checksum file.
+    return ((size + bytesPerSum - 1) / bytesPerSum) * FSInputChecker.CHECKSUM_SIZE +
+             ChecksumFSInputChecker.HEADER_LENGTH;
   }
 
   /** This class provides an output stream for a checksummed file.
    * It generates checksums for data. */
   private static class ChecksumFSOutputSummer extends FSOutputSummer
       implements IOStatisticsSource, StreamCapabilities {
-    private FSDataOutputStream datas;    
+    private FSDataOutputStream datas;
     private FSDataOutputStream sums;
     private static final float CHKSUM_AS_FRACTION = 0.01f;
     private boolean isClosed = false;
-    
-    public ChecksumFSOutputSummer(ChecksumFileSystem fs, 
-                          Path file, 
+
+    ChecksumFSOutputSummer(ChecksumFileSystem fs,
+                          Path file,
                           boolean overwrite,
                           int bufferSize,
                           short replication,
@@ -460,7 +590,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
       sums.writeInt(bytesPerSum);
     }
-    
+
     @Override
     public void close() throws IOException {
       try {
@@ -471,7 +601,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
         isClosed = true;
       }
     }
-    
+
     @Override
     protected void writeChunk(byte[] b, int offset, int len, byte[] checksum,
         int ckoff, int cklen)
@@ -727,7 +857,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
         value = fs.rename(srcCheckFile, dstCheckFile);
       } else if (fs.exists(dstCheckFile)) {
         // no src checksum, so remove dst checksum
-        value = fs.delete(dstCheckFile, true); 
+        value = fs.delete(dstCheckFile, true);
       }
 
       return value;
@@ -759,7 +889,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       return fs.delete(f, true);
     }
   }
-    
+
   final private static PathFilter DEFAULT_FILTER = new PathFilter() {
     @Override
     public boolean accept(Path file) {
@@ -770,7 +900,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   /**
    * List the statuses of the files/directories in the given path if the path is
    * a directory.
-   * 
+   *
    * @param f
    *          given path
    * @return the statuses of the files/directories in the given path
@@ -791,7 +921,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   /**
    * List the statuses of the files/directories in the given path if the path is
    * a directory.
-   * 
+   *
    * @param f
    *          given path
    * @return the statuses of the files/directories in the given patch
@@ -802,7 +932,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   throws IOException {
     return fs.listLocatedStatus(f, DEFAULT_FILTER);
   }
-  
+
   @Override
   public boolean mkdirs(Path f) throws IOException {
     return fs.mkdirs(f);
@@ -856,7 +986,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     } else {
       FileStatus[] srcs = listStatus(src);
       for (FileStatus srcFile : srcs) {
-        copyToLocalFile(srcFile.getPath(), 
+        copyToLocalFile(srcFile.getPath(),
                         new Path(dst, srcFile.getPath().getName()), copyCrc);
       }
     }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
index b143a4cb63d..52644402ca4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -26,6 +26,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.nio.ByteBuffer;
 import java.util.EnumSet;
+import java.util.List;
+import java.util.function.IntFunction;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -51,7 +53,7 @@ public class FSDataInputStream extends DataInputStream
    */
   private final IdentityHashStore<ByteBuffer, ByteBufferPool>
     extendedReadBuffers
-      = new IdentityHashStore<ByteBuffer, ByteBufferPool>(0);
+      = new IdentityHashStore<>(0);
 
   public FSDataInputStream(InputStream in) {
     super(in);
@@ -279,4 +281,20 @@ public class FSDataInputStream extends DataInputStream
   public IOStatistics getIOStatistics() {
     return IOStatisticsSupport.retrieveIOStatistics(in);
   }
+
+  @Override
+  public int minSeekForVectorReads() {
+    return ((PositionedReadable) in).minSeekForVectorReads();
+  }
+
+  @Override
+  public int maxReadSizeForVectorReads() {
+    return ((PositionedReadable) in).maxReadSizeForVectorReads();
+  }
+
+  @Override
+  public void readVectored(List<? extends FileRange> ranges,
+                           IntFunction<ByteBuffer> allocate) throws IOException {
+    ((PositionedReadable) in).readVectored(ranges, allocate);
+  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java
new file mode 100644
index 00000000000..7388e462cc2
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A byte range of a file.
+ * This is used for the asynchronous gather read API of
+ * {@link PositionedReadable#readVectored}.
+ */
+public interface FileRange {
+
+  /**
+   * Get the starting offset of the range.
+   * @return the byte offset of the start
+   */
+  long getOffset();
+
+  /**
+   * Get the length of the range.
+   * @return the number of bytes in the range.
+   */
+  int getLength();
+
+  /**
+   * Get the future data for this range.
+   * @return the future for the {@link ByteBuffer} that contains the data
+   */
+  CompletableFuture<ByteBuffer> getData();
+
+  /**
+   * Set a future for this range's data.
+   * This method is called by {@link PositionedReadable#readVectored} to store the
+   * data for the user to pick up later via {@link #getData}.
+   * @param data the future of the ByteBuffer that will have the data
+   */
+  void setData(CompletableFuture<ByteBuffer> data);
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRangeImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRangeImpl.java
new file mode 100644
index 00000000000..ef5851154be
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRangeImpl.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A range of bytes from a file with an optional buffer to read those bytes
+ * for zero copy.
+ */
+public class FileRangeImpl implements FileRange {
+  private long offset;
+  private int length;
+  private CompletableFuture<ByteBuffer> reader;
+
+  public FileRangeImpl(long offset, int length) {
+    this.offset = offset;
+    this.length = length;
+  }
+
+  @Override
+  public String toString() {
+    return "range[" + offset + "," + (offset + length) + ")";
+  }
+
+  @Override
+  public long getOffset() {
+    return offset;
+  }
+
+  @Override
+  public int getLength() {
+    return length;
+  }
+
+  public void setOffset(long offset) {
+    this.offset = offset;
+  }
+
+  public void setLength(int length) {
+    this.length = length;
+  }
+
+  @Override
+  public void setData(CompletableFuture<ByteBuffer> pReader) {
+    this.reader = pReader;
+  }
+
+  @Override
+  public CompletableFuture<ByteBuffer> getData() {
+    return reader;
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PositionedReadable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PositionedReadable.java
index 6744d17a726..7e543ebf226 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PositionedReadable.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PositionedReadable.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,10 +17,15 @@
  */
 package org.apache.hadoop.fs;
 
-import java.io.*;
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.function.IntFunction;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.impl.VectoredReadUtils;
 
 /**
  * Stream that permits positional reading.
@@ -85,4 +90,38 @@ public interface PositionedReadable {
    * the read operation completed
    */
   void readFully(long position, byte[] buffer) throws IOException;
+
+  /**
+   * What is the smallest reasonable seek?
+   * @return the minimum number of bytes
+   */
+  default int minSeekForVectorReads() {
+    return 4 * 1024;
+  }
+
+  /**
+   * What is the largest size that we should group ranges together as?
+   * @return the number of bytes to read at once
+   */
+  default int maxReadSizeForVectorReads() {
+    return 1024 * 1024;
+  }
+
+  /**
+   * Read fully a list of file ranges asynchronously from this file.
+   * The default iterates through the ranges to read each synchronously, but
+   * the intent is that FSDataInputStream subclasses can make more efficient
+   * readers.
+   * As a result of the call, each range will have FileRange.setData(CompletableFuture)
+   * called with a future that when complete will have a ByteBuffer with the
+   * data from the file's range.
+   * @param ranges the byte ranges to read
+   * @param allocate the function to allocate ByteBuffer
+   * @throws IOException any IOE.
+   */
+  default void readVectored(List<? extends FileRange> ranges,
+                            IntFunction<ByteBuffer> allocate) throws IOException {
+    VectoredReadUtils.readVectored(this, ranges, allocate,  minSeekForVectorReads(),
+        maxReadSizeForVectorReads());
+  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index 86e42f5d7cd..c71692e67e3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -20,6 +20,7 @@
 package org.apache.hadoop.fs;
 
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.impl.VectoredReadUtils;
 
 import java.io.BufferedOutputStream;
 import java.io.DataOutput;
@@ -33,8 +34,11 @@ import java.io.OutputStream;
 import java.io.FileDescriptor;
 import java.net.URI;
 import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousFileChannel;
+import java.nio.channels.CompletionHandler;
 import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
+import java.nio.file.StandardOpenOption;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.nio.file.attribute.BasicFileAttributeView;
 import java.nio.file.attribute.FileTime;
@@ -44,6 +48,9 @@ import java.util.Locale;
 import java.util.Optional;
 import java.util.StringTokenizer;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.IntFunction;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -130,7 +137,9 @@ public class RawLocalFileSystem extends FileSystem {
   class LocalFSFileInputStream extends FSInputStream implements
       HasFileDescriptor, IOStatisticsSource, StreamCapabilities {
     private FileInputStream fis;
+    private final File name;
     private long position;
+    private AsynchronousFileChannel asyncChannel = null;
 
     /**
      * Minimal set of counters.
@@ -148,7 +157,8 @@ public class RawLocalFileSystem extends FileSystem {
     private final AtomicLong bytesRead;
 
     public LocalFSFileInputStream(Path f) throws IOException {
-      fis = new FileInputStream(pathToFile(f));
+      name = pathToFile(f);
+      fis = new FileInputStream(name);
       bytesRead = ioStatistics.getCounterReference(
           STREAM_READ_BYTES);
     }
@@ -179,10 +189,16 @@ public class RawLocalFileSystem extends FileSystem {
     @Override
     public int available() throws IOException { return fis.available(); }
     @Override
-    public void close() throws IOException { fis.close(); }
-    @Override
     public boolean markSupported() { return false; }
-    
+
+    @Override
+    public void close() throws IOException {
+      fis.close();
+      if (asyncChannel != null) {
+        asyncChannel.close();
+      }
+    }
+
     @Override
     public int read() throws IOException {
       try {
@@ -272,8 +288,88 @@ public class RawLocalFileSystem extends FileSystem {
     public IOStatistics getIOStatistics() {
       return ioStatistics;
     }
+
+    AsynchronousFileChannel getAsyncChannel() throws IOException {
+      if (asyncChannel == null) {
+        synchronized (this) {
+          asyncChannel = AsynchronousFileChannel.open(name.toPath(),
+                  StandardOpenOption.READ);
+        }
+      }
+      return asyncChannel;
+    }
+
+    @Override
+    public void readVectored(List<? extends FileRange> ranges,
+                             IntFunction<ByteBuffer> allocate) throws IOException {
+
+      // Set up all of the futures, so that we can use them if things fail
+      for(FileRange range: ranges) {
+        VectoredReadUtils.validateRangeRequest(range);
+        range.setData(new CompletableFuture<>());
+      }
+      try {
+        AsynchronousFileChannel channel = getAsyncChannel();
+        ByteBuffer[] buffers = new ByteBuffer[ranges.size()];
+        AsyncHandler asyncHandler = new AsyncHandler(channel, ranges, buffers);
+        for(int i = 0; i < ranges.size(); ++i) {
+          FileRange range = ranges.get(i);
+          buffers[i] = allocate.apply(range.getLength());
+          channel.read(buffers[i], range.getOffset(), i, asyncHandler);
+        }
+      } catch (IOException ioe) {
+        LOG.debug("Exception occurred during vectored read ", ioe);
+        for(FileRange range: ranges) {
+          range.getData().completeExceptionally(ioe);
+        }
+      }
+    }
   }
-  
+
+  /**
+   * A CompletionHandler that implements readFully and translates back
+   * into the form of CompletionHandler that our users expect.
+   */
+  static class AsyncHandler implements CompletionHandler<Integer, Integer> {
+    private final AsynchronousFileChannel channel;
+    private final List<? extends FileRange> ranges;
+    private final ByteBuffer[] buffers;
+
+    AsyncHandler(AsynchronousFileChannel channel,
+                 List<? extends FileRange> ranges,
+                 ByteBuffer[] buffers) {
+      this.channel = channel;
+      this.ranges = ranges;
+      this.buffers = buffers;
+    }
+
+    @Override
+    public void completed(Integer result, Integer r) {
+      FileRange range = ranges.get(r);
+      ByteBuffer buffer = buffers[r];
+      if (result == -1) {
+        failed(new EOFException("Read past End of File"), r);
+      } else {
+        if (buffer.remaining() > 0) {
+          // issue a read for the rest of the buffer
+          // QQ: What if this fails? It has the same handler.
+          channel.read(buffer, range.getOffset() + buffer.position(), r, this);
+        } else {
+          // QQ: Why  is this required? I think because we don't want the
+          // user to read data beyond limit.
+          buffer.flip();
+          range.getData().complete(buffer);
+        }
+      }
+    }
+
+    @Override
+    public void failed(Throwable exc, Integer r) {
+      LOG.debug("Failed while reading range " + r + " {} ", exc);
+      ranges.get(r).getData().completeExceptionally(exc);
+    }
+  }
+
   @Override
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
     getFileStatus(f);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java
new file mode 100644
index 00000000000..828a50b4f7b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.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;
+
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.FileRangeImpl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A file range that represents a set of underlying file ranges.
+ * This is used when we combine the user's FileRange objects
+ * together into a single read for efficiency.
+ */
+public class CombinedFileRange extends FileRangeImpl {
+  private ArrayList<FileRange> underlying = new ArrayList<>();
+
+  public CombinedFileRange(long offset, long end, FileRange original) {
+    super(offset, (int) (end - offset));
+    this.underlying.add(original);
+  }
+
+  /**
+   * Get the list of ranges that were merged together to form this one.
+   * @return the list of input ranges
+   */
+  public List<FileRange> getUnderlying() {
+    return underlying;
+  }
+
+  /**
+   * Merge this input range into the current one, if it is compatible.
+   * It is assumed that otherOffset is greater or equal the current offset,
+   * which typically happens by sorting the input ranges on offset.
+   * @param otherOffset the offset to consider merging
+   * @param otherEnd the end to consider merging
+   * @param other the underlying FileRange to add if we merge
+   * @param minSeek the minimum distance that we'll seek without merging the
+   *                ranges together
+   * @param maxSize the maximum size that we'll merge into a single range
+   * @return true if we have merged the range into this one
+   */
+  public boolean merge(long otherOffset, long otherEnd, FileRange other,
+                       int minSeek, int maxSize) {
+    long end = this.getOffset() + this.getLength();
+    long newEnd = Math.max(end, otherEnd);
+    if (otherOffset - end >= minSeek || newEnd - this.getOffset() > maxSize) {
+      return false;
+    }
+    this.setLength((int) (newEnd - this.getOffset()));
+    underlying.add(other);
+    return true;
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/VectoredReadUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/VectoredReadUtils.java
new file mode 100644
index 00000000000..9a16e6841dd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/VectoredReadUtils.java
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.IntFunction;
+
+import org.apache.hadoop.fs.ByteBufferPositionedReadable;
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.util.Preconditions;
+
+/**
+ * Utility class which implements helper methods used
+ * in vectored IO implementation.
+ */
+public final class VectoredReadUtils {
+
+  /**
+   * Validate a single range.
+   * @param range file range.
+   * @throws EOFException any EOF Exception.
+   */
+  public static void validateRangeRequest(FileRange range)
+          throws EOFException {
+
+    Preconditions.checkArgument(range.getLength() >= 0, "length is negative");
+    if (range.getOffset() < 0) {
+      throw new EOFException("position is negative");
+    }
+  }
+
+  /**
+   * Validate a list of vectored read ranges.
+   * @param ranges list of ranges.
+   * @throws EOFException any EOF exception.
+   */
+  public static void validateVectoredReadRanges(List<? extends FileRange> ranges)
+          throws EOFException {
+    for (FileRange range : ranges) {
+      validateRangeRequest(range);
+    }
+  }
+
+
+
+  /**
+   * Read fully a list of file ranges asynchronously from this file.
+   * The default iterates through the ranges to read each synchronously, but
+   * the intent is that subclasses can make more efficient readers.
+   * The data or exceptions are pushed into {@link FileRange#getData()}.
+   * @param stream the stream to read the data from
+   * @param ranges the byte ranges to read
+   * @param allocate the byte buffer allocation
+   * @param minimumSeek the minimum number of bytes to seek over
+   * @param maximumRead the largest number of bytes to combine into a single read
+   */
+  public static void readVectored(PositionedReadable stream,
+                                  List<? extends FileRange> ranges,
+                                  IntFunction<ByteBuffer> allocate,
+                                  int minimumSeek,
+                                  int maximumRead) {
+    if (isOrderedDisjoint(ranges, 1, minimumSeek)) {
+      for(FileRange range: ranges) {
+        range.setData(readRangeFrom(stream, range, allocate));
+      }
+    } else {
+      for(CombinedFileRange range: sortAndMergeRanges(ranges, 1, minimumSeek,
+          maximumRead)) {
+        CompletableFuture<ByteBuffer> read =
+            readRangeFrom(stream, range, allocate);
+        for(FileRange child: range.getUnderlying()) {
+          child.setData(read.thenApply(
+              (b) -> sliceTo(b, range.getOffset(), child)));
+        }
+      }
+    }
+  }
+
+  /**
+   * Synchronously reads a range from the stream dealing with the combinations
+   * of ByteBuffers buffers and PositionedReadable streams.
+   * @param stream the stream to read from
+   * @param range the range to read
+   * @param allocate the function to allocate ByteBuffers
+   * @return the CompletableFuture that contains the read data
+   */
+  public static CompletableFuture<ByteBuffer> readRangeFrom(PositionedReadable stream,
+                                                            FileRange range,
+                                                            IntFunction<ByteBuffer> allocate) {
+    CompletableFuture<ByteBuffer> result = new CompletableFuture<>();
+    try {
+      ByteBuffer buffer = allocate.apply(range.getLength());
+      if (stream instanceof ByteBufferPositionedReadable) {
+        ((ByteBufferPositionedReadable) stream).readFully(range.getOffset(),
+            buffer);
+        buffer.flip();
+      } else {
+        readNonByteBufferPositionedReadable(stream, range, buffer);
+      }
+      result.complete(buffer);
+    } catch (IOException ioe) {
+      result.completeExceptionally(ioe);
+    }
+    return result;
+  }
+
+  private static void readNonByteBufferPositionedReadable(PositionedReadable stream,
+                                                          FileRange range,
+                                                          ByteBuffer buffer) throws IOException {
+    if (buffer.isDirect()) {
+      buffer.put(readInDirectBuffer(stream, range));
+      buffer.flip();
+    } else {
+      stream.readFully(range.getOffset(), buffer.array(),
+              buffer.arrayOffset(), range.getLength());
+    }
+  }
+
+  private static byte[] readInDirectBuffer(PositionedReadable stream,
+                                           FileRange range) throws IOException {
+    // if we need to read data from a direct buffer and the stream doesn't
+    // support it, we allocate a byte array to use.
+    byte[] tmp = new byte[range.getLength()];
+    stream.readFully(range.getOffset(), tmp, 0, tmp.length);
+    return tmp;
+  }
+
+  /**
+   * Is the given input list.
+   * <ul>
+   *   <li>already sorted by offset</li>
+   *   <li>each range is more than minimumSeek apart</li>
+   *   <li>the start and end of each range is a multiple of chunkSize</li>
+   * </ul>
+   *
+   * @param input the list of input ranges.
+   * @param chunkSize the size of the chunks that the offset and end must align to.
+   * @param minimumSeek the minimum distance between ranges.
+   * @return true if we can use the input list as is.
+   */
+  public static boolean isOrderedDisjoint(List<? extends FileRange> input,
+                                          int chunkSize,
+                                          int minimumSeek) {
+    long previous = -minimumSeek;
+    for(FileRange range: input) {
+      long offset = range.getOffset();
+      long end = range.getOffset() + range.getLength();
+      if (offset % chunkSize != 0 ||
+              end % chunkSize != 0 ||
+              (offset - previous < minimumSeek)) {
+        return false;
+      }
+      previous = end;
+    }
+    return true;
+  }
+
+  /**
+   * Calculates floor value of offset based on chunk size.
+   * @param offset file offset.
+   * @param chunkSize file chunk size.
+   * @return  floor value.
+   */
+  public static long roundDown(long offset, int chunkSize) {
+    if (chunkSize > 1) {
+      return offset - (offset % chunkSize);
+    } else {
+      return offset;
+    }
+  }
+
+  /**
+   * Calculates the ceil value of offset based on chunk size.
+   * @param offset file offset.
+   * @param chunkSize file chunk size.
+   * @return ceil value.
+   */
+  public static long roundUp(long offset, int chunkSize) {
+    if (chunkSize > 1) {
+      long next = offset + chunkSize - 1;
+      return next - (next % chunkSize);
+    } else {
+      return offset;
+    }
+  }
+
+  /**
+   * Sort and merge ranges to optimize the access from the underlying file
+   * system.
+   * The motivations are that:
+   * <ul>
+   *   <li>Upper layers want to pass down logical file ranges.</li>
+   *   <li>Fewer reads have better performance.</li>
+   *   <li>Applications want callbacks as ranges are read.</li>
+   *   <li>Some file systems want to round ranges to be at checksum boundaries.</li>
+   * </ul>
+   *
+   * @param input the list of input ranges
+   * @param chunkSize round the start and end points to multiples of chunkSize
+   * @param minimumSeek the smallest gap that we should seek over in bytes
+   * @param maxSize the largest combined file range in bytes
+   * @return the list of sorted CombinedFileRanges that cover the input
+   */
+  public static List<CombinedFileRange> sortAndMergeRanges(List<? extends FileRange> input,
+                                                           int chunkSize,
+                                                           int minimumSeek,
+                                                           int maxSize) {
+    // sort the ranges by offset
+    FileRange[] ranges = input.toArray(new FileRange[0]);
+    Arrays.sort(ranges, Comparator.comparingLong(FileRange::getOffset));
+    CombinedFileRange current = null;
+    List<CombinedFileRange> result = new ArrayList<>(ranges.length);
+
+    // now merge together the ones that merge
+    for(FileRange range: ranges) {
+      long start = roundDown(range.getOffset(), chunkSize);
+      long end = roundUp(range.getOffset() + range.getLength(), chunkSize);
+      if (current == null || !current.merge(start, end, range, minimumSeek, maxSize)) {
+        current = new CombinedFileRange(start, end, range);
+        result.add(current);
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Slice the data that was read to the user's request.
+   * This function assumes that the user's request is completely subsumed by the
+   * read data. This always creates a new buffer pointing to the same underlying
+   * data but with its own mark and position fields such that reading one buffer
+   * can't effect other's mark and position.
+   * @param readData the buffer with the readData
+   * @param readOffset the offset in the file for the readData
+   * @param request the user's request
+   * @return the readData buffer that is sliced to the user's request
+   */
+  public static ByteBuffer sliceTo(ByteBuffer readData, long readOffset,
+                                   FileRange request) {
+    int offsetChange = (int) (request.getOffset() - readOffset);
+    int requestLength = request.getLength();
+    readData = readData.slice();
+    readData.position(offsetChange);
+    readData.limit(offsetChange + requestLength);
+    return readData;
+  }
+
+  /**
+   * private constructor.
+   */
+  private VectoredReadUtils() {
+    throw new UnsupportedOperationException();
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
index 090696483be..0fe1772d266 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
@@ -443,6 +443,37 @@ The semantics of this are exactly equivalent to
 That is, the buffer is filled entirely with the contents of the input source
 from position `position`
 
+### `default void readVectored(List<? extends FileRange> ranges, IntFunction<ByteBuffer> allocate)`
+
+Read fully data for a list of ranges asynchronously. The default implementation
+iterates through the ranges, tries to coalesce the ranges based on values of
+`minSeekForVectorReads` and `maxReadSizeForVectorReads` and then read each merged
+ranges synchronously, but the intent is sub classes can implement efficient
+implementation.
+
+#### Preconditions
+
+For each requested range:
+
+    range.getOffset >= 0 else raise IllegalArgumentException
+    range.getLength >= 0 else raise EOFException
+
+#### Postconditions
+
+For each requested range:
+
+    range.getData() returns CompletableFuture<ByteBuffer> which will have data
+    from range.getOffset to range.getLength.
+
+### `minSeekForVectorReads()`
+
+Smallest reasonable seek. Two ranges won't be merged together if the difference between
+end of first and start of next range is more than this value.
+
+### `maxReadSizeForVectorReads()`
+
+Maximum number of bytes which can be read in one go after merging the ranges.
+Two ranges won't be merged if the combined data to be read is more than this value.
 
 ## Consistency
 
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
new file mode 100644
index 00000000000..eee4b11e739
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.FileRangeImpl;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.impl.FutureIOSupport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.IntFunction;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+
+@RunWith(Parameterized.class)
+public abstract class AbstractContractVectoredReadTest extends AbstractFSContractTestBase {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbstractContractVectoredReadTest.class);
+
+  public static final int DATASET_LEN = 64 * 1024;
+  private static final byte[] DATASET = ContractTestUtils.dataset(DATASET_LEN, 'a', 32);
+  private static final String VECTORED_READ_FILE_NAME = "vectored_file.txt";
+  private static final String VECTORED_READ_FILE_1MB_NAME = "vectored_file_1M.txt";
+  private static final byte[] DATASET_MB = ContractTestUtils.dataset(1024 * 1024, 'a', 256);
+
+  private final IntFunction<ByteBuffer> allocate;
+
+  private final String bufferType;
+
+  @Parameterized.Parameters(name = "Buffer type : {0}")
+  public static List<String> params() {
+    return Arrays.asList("direct", "array");
+  }
+
+  public AbstractContractVectoredReadTest(String bufferType) {
+    this.bufferType = bufferType;
+    this.allocate = "array".equals(bufferType) ?
+            ByteBuffer::allocate : ByteBuffer::allocateDirect;
+  }
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    Path path = path(VECTORED_READ_FILE_NAME);
+    FileSystem fs = getFileSystem();
+    createFile(fs, path, true, DATASET);
+    Path bigFile = path(VECTORED_READ_FILE_1MB_NAME);
+    createFile(fs, bigFile, true, DATASET_MB);
+  }
+
+  @Test
+  public void testVectoredReadMultipleRanges() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      FileRange fileRange = new FileRangeImpl(i * 100, 100);
+      fileRanges.add(fileRange);
+    }
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      CompletableFuture<?>[] completableFutures = new CompletableFuture<?>[fileRanges.size()];
+      int i = 0;
+      for (FileRange res : fileRanges) {
+        completableFutures[i++] = res.getData();
+      }
+      CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(completableFutures);
+      combinedFuture.get();
+
+      validateVectoredReadResult(fileRanges);
+    }
+  }
+
+  @Test
+  public void testVectoredReadAndReadFully()  throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(100, 100));
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      byte[] readFullRes = new byte[100];
+      in.readFully(100, readFullRes);
+      ByteBuffer vecRes = FutureIOSupport.awaitFuture(fileRanges.get(0).getData());
+      Assertions.assertThat(vecRes)
+              .describedAs("Result from vectored read and readFully must match")
+              .isEqualByComparingTo(ByteBuffer.wrap(readFullRes));
+    }
+  }
+
+  /**
+   * As the minimum seek value is 4*1024,none of the below ranges
+   * will get merged.
+   */
+  @Test
+  public void testDisjointRanges() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(0, 100));
+    fileRanges.add(new FileRangeImpl(4 * 1024 + 101, 100));
+    fileRanges.add(new FileRangeImpl(16 * 1024 + 101, 100));
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      validateVectoredReadResult(fileRanges);
+    }
+  }
+
+  /**
+   * As the minimum seek value is 4*1024, all the below ranges
+   * will get merged into one.
+   */
+  @Test
+  public void testAllRangesMergedIntoOne() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(0, 100));
+    fileRanges.add(new FileRangeImpl(4 *1024 - 101, 100));
+    fileRanges.add(new FileRangeImpl(8*1024 - 101, 100));
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      validateVectoredReadResult(fileRanges);
+    }
+  }
+
+  /**
+   * As the minimum seek value is 4*1024, the first three ranges will be
+   * merged into and other two will remain as it is.
+   */
+  @Test
+  public void testSomeRangesMergedSomeUnmerged() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(8*1024, 100));
+    fileRanges.add(new FileRangeImpl(14*1024, 100));
+    fileRanges.add(new FileRangeImpl(10*1024, 100));
+    fileRanges.add(new FileRangeImpl(2 *1024 - 101, 100));
+    fileRanges.add(new FileRangeImpl(40*1024, 1024));
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      validateVectoredReadResult(fileRanges);
+    }
+  }
+
+  public void testSameRanges() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(8*1024, 1000));
+    fileRanges.add(new FileRangeImpl(8*1024, 1000));
+    fileRanges.add(new FileRangeImpl(8*1024, 1000));
+    CompletableFuture<FSDataInputStream> builder =
+            fs.openFile(path(VECTORED_READ_FILE_NAME))
+                    .build();
+    try (FSDataInputStream in = builder.get()) {
+      in.readVectored(fileRanges, allocate);
+      validateVectoredReadResult(fileRanges);
+    }
+  }
+
+  @Test
+  public void testVectoredRead1MBFile()  throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(1293, 25837));
+    CompletableFuture<FSDataInputStream> builder =
+            fs.openFile(path(VECTORED_READ_FILE_1MB_NAME))
+            .build();
+    try (FSDataInputStream in = builder.get()) {
+      in.readVectored(fileRanges, allocate);
+      ByteBuffer vecRes = FutureIOSupport.awaitFuture(fileRanges.get(0).getData());
+      FileRange resRange = fileRanges.get(0);
+      assertDatasetEquals((int) resRange.getOffset(), "vecRead",
+              vecRes, resRange.getLength(), DATASET_MB);
+    }
+  }
+
+  @Test
+  public void testOverlappingRanges() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(0, 1000));
+    fileRanges.add(new FileRangeImpl(90, 900));
+    fileRanges.add(new FileRangeImpl(50, 900));
+    fileRanges.add(new FileRangeImpl(10, 980));
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      validateVectoredReadResult(fileRanges);
+    }
+  }
+
+  @Test
+  public void testEOFRanges()  throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(DATASET_LEN, 100));
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      for (FileRange res : fileRanges) {
+        CompletableFuture<ByteBuffer> data = res.getData();
+        try {
+          ByteBuffer buffer = data.get();
+          // Shouldn't reach here.
+          Assert.fail("EOFException must be thrown while reading EOF");
+        } catch (ExecutionException ex) {
+          // ignore as expected.
+        } catch (Exception ex) {
+          LOG.error("Exception while running vectored read ", ex);
+          Assert.fail("Exception while running vectored read " + ex);
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testNegativeLengthRange()  throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(0, -50));
+    testExceptionalVectoredRead(fs, fileRanges, "Exception is expected");
+  }
+
+  @Test
+  public void testNegativeOffsetRange()  throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(-1, 50));
+    testExceptionalVectoredRead(fs, fileRanges, "Exception is expected");
+  }
+
+  @Test
+  public void testNormalReadAfterVectoredRead() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = createSomeOverlappingRanges();
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      // read starting 200 bytes
+      byte[] res = new byte[200];
+      in.read(res, 0, 200);
+      ByteBuffer buffer = ByteBuffer.wrap(res);
+      assertDatasetEquals(0, "normal_read", buffer, 200, DATASET);
+      Assertions.assertThat(in.getPos())
+              .describedAs("Vectored read shouldn't change file pointer.")
+              .isEqualTo(200);
+      validateVectoredReadResult(fileRanges);
+    }
+  }
+
+  @Test
+  public void testVectoredReadAfterNormalRead() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = createSomeOverlappingRanges();
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      // read starting 200 bytes
+      byte[] res = new byte[200];
+      in.read(res, 0, 200);
+      ByteBuffer buffer = ByteBuffer.wrap(res);
+      assertDatasetEquals(0, "normal_read", buffer, 200, DATASET);
+      Assertions.assertThat(in.getPos())
+              .describedAs("Vectored read shouldn't change file pointer.")
+              .isEqualTo(200);
+      in.readVectored(fileRanges, allocate);
+      validateVectoredReadResult(fileRanges);
+    }
+  }
+
+  @Test
+  public void testMultipleVectoredReads() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges1 = createSomeOverlappingRanges();
+    List<FileRange> fileRanges2 = createSomeOverlappingRanges();
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges1, allocate);
+      in.readVectored(fileRanges2, allocate);
+      validateVectoredReadResult(fileRanges2);
+      validateVectoredReadResult(fileRanges1);
+    }
+  }
+
+  protected List<FileRange> createSomeOverlappingRanges() {
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(0, 100));
+    fileRanges.add(new FileRangeImpl(90, 50));
+    return fileRanges;
+  }
+
+  protected void validateVectoredReadResult(List<FileRange> fileRanges)
+          throws ExecutionException, InterruptedException {
+    CompletableFuture<?>[] completableFutures = new CompletableFuture<?>[fileRanges.size()];
+    int i = 0;
+    for (FileRange res : fileRanges) {
+      completableFutures[i++] = res.getData();
+    }
+    CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(completableFutures);
+    combinedFuture.get();
+
+    for (FileRange res : fileRanges) {
+      CompletableFuture<ByteBuffer> data = res.getData();
+      try {
+        ByteBuffer buffer = FutureIOSupport.awaitFuture(data);
+        assertDatasetEquals((int) res.getOffset(), "vecRead", buffer, res.getLength(), DATASET);
+      } catch (Exception ex) {
+        LOG.error("Exception while running vectored read ", ex);
+        Assert.fail("Exception while running vectored read " + ex);
+      }
+    }
+  }
+
+  protected void testExceptionalVectoredRead(FileSystem fs,
+                                             List<FileRange> fileRanges,
+                                             String s) throws IOException {
+    boolean exRaised = false;
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      // Can we intercept here as done in S3 tests ??
+      in.readVectored(fileRanges, allocate);
+    } catch (EOFException | IllegalArgumentException ex) {
+      // expected.
+      exRaised = true;
+    }
+    Assertions.assertThat(exRaised)
+            .describedAs(s)
+            .isTrue();
+  }
+
+  /**
+   * Assert that the data read matches the dataset at the given offset.
+   * This helps verify that the seek process is moving the read pointer
+   * to the correct location in the file.
+   *  @param readOffset the offset in the file where the read began.
+   * @param operation  operation name for the assertion.
+   * @param data       data read in.
+   * @param length     length of data to check.
+   * @param originalData
+   */
+  private void assertDatasetEquals(
+          final int readOffset, final String operation,
+          final ByteBuffer data,
+          int length, byte[] originalData) {
+    for (int i = 0; i < length; i++) {
+      int o = readOffset + i;
+      assertEquals(operation + " with read offset " + readOffset
+                      + ": data[" + i + "] != DATASET[" + o + "]",
+              originalData[o], data.get());
+    }
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java
new file mode 100644
index 00000000000..099e3b946d1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestLocalFSContractVectoredRead extends AbstractContractVectoredReadTest {
+
+  public TestLocalFSContractVectoredRead(String bufferType) {
+    super(bufferType);
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractVectoredRead.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractVectoredRead.java
new file mode 100644
index 00000000000..cbb31ffe27a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractVectoredRead.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestRawLocalContractVectoredRead extends AbstractContractVectoredReadTest {
+
+  public TestRawLocalContractVectoredRead(String bufferType) {
+    super(bufferType);
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RawlocalFSContract(conf);
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java
new file mode 100644
index 00000000000..f789f361905
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java
@@ -0,0 +1,344 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.IntBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.IntFunction;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.ArgumentMatchers;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.ByteBufferPositionedReadable;
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.FileRangeImpl;
+import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.test.HadoopTestBase;
+
+import static org.apache.hadoop.test.MoreAsserts.assertFutureCompletedSuccessfully;
+import static org.apache.hadoop.test.MoreAsserts.assertFutureFailedExceptionally;
+
+/**
+ * Test behavior of {@link VectoredReadUtils}.
+ */
+public class TestVectoredReadUtils extends HadoopTestBase {
+
+  @Test
+  public void testSliceTo() {
+    final int size = 64 * 1024;
+    ByteBuffer buffer = ByteBuffer.allocate(size);
+    // fill the buffer with data
+    IntBuffer intBuffer = buffer.asIntBuffer();
+    for(int i=0; i < size / Integer.BYTES; ++i) {
+      intBuffer.put(i);
+    }
+    // ensure we don't make unnecessary slices
+    ByteBuffer slice = VectoredReadUtils.sliceTo(buffer, 100,
+        new FileRangeImpl(100, size));
+    Assertions.assertThat(buffer)
+            .describedAs("Slicing on the same offset shouldn't " +
+                    "create a new buffer")
+            .isEqualTo(slice);
+
+    // try slicing a range
+    final int offset = 100;
+    final int sliceStart = 1024;
+    final int sliceLength = 16 * 1024;
+    slice = VectoredReadUtils.sliceTo(buffer, offset,
+        new FileRangeImpl(offset + sliceStart, sliceLength));
+    // make sure they aren't the same, but use the same backing data
+    Assertions.assertThat(buffer)
+            .describedAs("Slicing on new offset should " +
+                    "create a new buffer")
+            .isNotEqualTo(slice);
+    Assertions.assertThat(buffer.array())
+            .describedAs("Slicing should use the same underlying " +
+                    "data")
+            .isEqualTo(slice.array());
+    // test the contents of the slice
+    intBuffer = slice.asIntBuffer();
+    for(int i=0; i < sliceLength / Integer.BYTES; ++i) {
+      assertEquals("i = " + i, i + sliceStart / Integer.BYTES, intBuffer.get());
+    }
+  }
+
+  @Test
+  public void testRounding() {
+    for(int i=5; i < 10; ++i) {
+      assertEquals("i = "+ i, 5, VectoredReadUtils.roundDown(i, 5));
+      assertEquals("i = "+ i, 10, VectoredReadUtils.roundUp(i+1, 5));
+    }
+    assertEquals("Error while roundDown", 13, VectoredReadUtils.roundDown(13, 1));
+    assertEquals("Error while roundUp", 13, VectoredReadUtils.roundUp(13, 1));
+  }
+
+  @Test
+  public void testMerge() {
+    FileRange base = new FileRangeImpl(2000, 1000);
+    CombinedFileRange mergeBase = new CombinedFileRange(2000, 3000, base);
+
+    // test when the gap between is too big
+    assertFalse("Large gap ranges shouldn't get merged", mergeBase.merge(5000, 6000,
+        new FileRangeImpl(5000, 1000), 2000, 4000));
+    assertEquals("Number of ranges in merged range shouldn't increase",
+            1, mergeBase.getUnderlying().size());
+    assertEquals("post merge offset", 2000, mergeBase.getOffset());
+    assertEquals("post merge length", 1000, mergeBase.getLength());
+
+    // test when the total size gets exceeded
+    assertFalse("Large size ranges shouldn't get merged", mergeBase.merge(5000, 6000,
+        new FileRangeImpl(5000, 1000), 2001, 3999));
+    assertEquals("Number of ranges in merged range shouldn't increase",
+            1, mergeBase.getUnderlying().size());
+    assertEquals("post merge offset", 2000, mergeBase.getOffset());
+    assertEquals("post merge length", 1000, mergeBase.getLength());
+
+    // test when the merge works
+    assertTrue("ranges should get merged ", mergeBase.merge(5000, 6000,
+        new FileRangeImpl(5000, 1000), 2001, 4000));
+    assertEquals("post merge size", 2, mergeBase.getUnderlying().size());
+    assertEquals("post merge offset", 2000, mergeBase.getOffset());
+    assertEquals("post merge length", 4000, mergeBase.getLength());
+
+    // reset the mergeBase and test with a 10:1 reduction
+    mergeBase = new CombinedFileRange(200, 300, base);
+    assertEquals(200, mergeBase.getOffset());
+    assertEquals(100, mergeBase.getLength());
+    assertTrue("ranges should get merged ", mergeBase.merge(500, 600,
+        new FileRangeImpl(5000, 1000), 201, 400));
+    assertEquals("post merge size", 2, mergeBase.getUnderlying().size());
+    assertEquals("post merge offset", 200, mergeBase.getOffset());
+    assertEquals("post merge length", 400, mergeBase.getLength());
+  }
+
+  @Test
+  public void testSortAndMerge() {
+    List<FileRange> input = Arrays.asList(
+        new FileRangeImpl(3000, 100),
+        new FileRangeImpl(2100, 100),
+        new FileRangeImpl(1000, 100)
+        );
+    assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
+    List<CombinedFileRange> outputList = VectoredReadUtils.sortAndMergeRanges(
+        input, 100, 1001, 2500);
+    assertEquals("merged range size", 1, outputList.size());
+    CombinedFileRange output = outputList.get(0);
+    assertEquals("merged range underlying size", 3, output.getUnderlying().size());
+    assertEquals("range[1000,3100)", output.toString());
+    assertTrue("merged output ranges are disjoint",
+            VectoredReadUtils.isOrderedDisjoint(outputList, 100, 800));
+
+    // the minSeek doesn't allow the first two to merge
+    assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 100, 1000));
+    outputList = VectoredReadUtils.sortAndMergeRanges(input, 100, 1000, 2100);
+    assertEquals("merged range size", 2, outputList.size());
+    assertEquals("range[1000,1100)", outputList.get(0).toString());
+    assertEquals("range[2100,3100)", outputList.get(1).toString());
+    assertTrue("merged output ranges are disjoint",
+            VectoredReadUtils.isOrderedDisjoint(outputList, 100, 1000));
+
+    // the maxSize doesn't allow the third range to merge
+    assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
+    outputList = VectoredReadUtils.sortAndMergeRanges(input, 100, 1001, 2099);
+    assertEquals("merged range size", 2, outputList.size());
+    assertEquals("range[1000,2200)", outputList.get(0).toString());
+    assertEquals("range[3000,3100)", outputList.get(1).toString());
+    assertTrue("merged output ranges are disjoint",
+            VectoredReadUtils.isOrderedDisjoint(outputList, 100, 800));
+
+    // test the round up and round down (the maxSize doesn't allow any merges)
+    assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 16, 700));
+    outputList = VectoredReadUtils.sortAndMergeRanges(input, 16, 1001, 100);
+    assertEquals("merged range size", 3, outputList.size());
+    assertEquals("range[992,1104)", outputList.get(0).toString());
+    assertEquals("range[2096,2208)", outputList.get(1).toString());
+    assertEquals("range[2992,3104)", outputList.get(2).toString());
+    assertTrue("merged output ranges are disjoint",
+            VectoredReadUtils.isOrderedDisjoint(outputList, 16, 700));
+  }
+
+  @Test
+  public void testSortAndMergeMoreCases() throws Exception {
+    List<FileRange> input = Arrays.asList(
+            new FileRangeImpl(3000, 110),
+            new FileRangeImpl(3000, 100),
+            new FileRangeImpl(2100, 100),
+            new FileRangeImpl(1000, 100)
+    );
+    assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
+    List<CombinedFileRange> outputList = VectoredReadUtils.sortAndMergeRanges(
+            input, 1, 1001, 2500);
+    assertEquals("merged range size", 1, outputList.size());
+    CombinedFileRange output = outputList.get(0);
+    assertEquals("merged range underlying size", 4, output.getUnderlying().size());
+    assertEquals("range[1000,3110)", output.toString());
+    assertTrue("merged output ranges are disjoint",
+            VectoredReadUtils.isOrderedDisjoint(outputList, 1, 800));
+
+    outputList = VectoredReadUtils.sortAndMergeRanges(
+            input, 100, 1001, 2500);
+    assertEquals("merged range size", 1, outputList.size());
+    output = outputList.get(0);
+    assertEquals("merged range underlying size", 4, output.getUnderlying().size());
+    assertEquals("range[1000,3200)", output.toString());
+    assertTrue("merged output ranges are disjoint",
+            VectoredReadUtils.isOrderedDisjoint(outputList, 1, 800));
+
+  }
+  interface Stream extends PositionedReadable, ByteBufferPositionedReadable {
+    // nothing
+  }
+
+  static void fillBuffer(ByteBuffer buffer) {
+    byte b = 0;
+    while (buffer.remaining() > 0) {
+      buffer.put(b++);
+    }
+  }
+
+  @Test
+  public void testReadRangeFromByteBufferPositionedReadable() throws Exception {
+    Stream stream = Mockito.mock(Stream.class);
+    Mockito.doAnswer(invocation -> {
+      fillBuffer(invocation.getArgument(1));
+      return null;
+    }).when(stream).readFully(ArgumentMatchers.anyLong(),
+                              ArgumentMatchers.any(ByteBuffer.class));
+    CompletableFuture<ByteBuffer> result =
+        VectoredReadUtils.readRangeFrom(stream, new FileRangeImpl(1000, 100),
+        ByteBuffer::allocate);
+    assertFutureCompletedSuccessfully(result);
+    ByteBuffer buffer = result.get();
+    assertEquals("Size of result buffer", 100, buffer.remaining());
+    byte b = 0;
+    while (buffer.remaining() > 0) {
+      assertEquals("remain = " + buffer.remaining(), b++, buffer.get());
+    }
+
+    // test an IOException
+    Mockito.reset(stream);
+    Mockito.doThrow(new IOException("foo"))
+        .when(stream).readFully(ArgumentMatchers.anyLong(),
+                                ArgumentMatchers.any(ByteBuffer.class));
+    result =
+        VectoredReadUtils.readRangeFrom(stream, new FileRangeImpl(1000, 100),
+            ByteBuffer::allocate);
+    assertFutureFailedExceptionally(result);
+  }
+
+  static void runReadRangeFromPositionedReadable(IntFunction<ByteBuffer> allocate)
+          throws Exception {
+    PositionedReadable stream = Mockito.mock(PositionedReadable.class);
+    Mockito.doAnswer(invocation -> {
+      byte b=0;
+      byte[] buffer = invocation.getArgument(1);
+      for(int i=0; i < buffer.length; ++i) {
+        buffer[i] = b++;
+      }
+      return null;
+    }).when(stream).readFully(ArgumentMatchers.anyLong(),
+        ArgumentMatchers.any(), ArgumentMatchers.anyInt(),
+        ArgumentMatchers.anyInt());
+    CompletableFuture<ByteBuffer> result =
+        VectoredReadUtils.readRangeFrom(stream, new FileRangeImpl(1000, 100),
+            allocate);
+    assertFutureCompletedSuccessfully(result);
+    ByteBuffer buffer = result.get();
+    assertEquals("Size of result buffer", 100, buffer.remaining());
+    byte b = 0;
+    while (buffer.remaining() > 0) {
+      assertEquals("remain = " + buffer.remaining(), b++, buffer.get());
+    }
+
+    // test an IOException
+    Mockito.reset(stream);
+    Mockito.doThrow(new IOException("foo"))
+        .when(stream).readFully(ArgumentMatchers.anyLong(),
+        ArgumentMatchers.any(), ArgumentMatchers.anyInt(),
+        ArgumentMatchers.anyInt());
+    result =
+        VectoredReadUtils.readRangeFrom(stream, new FileRangeImpl(1000, 100),
+            ByteBuffer::allocate);
+    assertFutureFailedExceptionally(result);
+  }
+
+  @Test
+  public void testReadRangeArray() throws Exception {
+    runReadRangeFromPositionedReadable(ByteBuffer::allocate);
+  }
+
+  @Test
+  public void testReadRangeDirect() throws Exception {
+    runReadRangeFromPositionedReadable(ByteBuffer::allocateDirect);
+  }
+
+  static void validateBuffer(String message, ByteBuffer buffer, int start) {
+    byte expected = (byte) start;
+    while (buffer.remaining() > 0) {
+      assertEquals(message + " remain: " + buffer.remaining(), expected++,
+          buffer.get());
+    }
+  }
+
+  @Test
+  public void testReadVectored() throws Exception {
+    List<FileRange> input = Arrays.asList(new FileRangeImpl(0, 100),
+        new FileRangeImpl(100_000, 100),
+        new FileRangeImpl(200_000, 100));
+    Stream stream = Mockito.mock(Stream.class);
+    Mockito.doAnswer(invocation -> {
+      fillBuffer(invocation.getArgument(1));
+      return null;
+    }).when(stream).readFully(ArgumentMatchers.anyLong(),
+        ArgumentMatchers.any(ByteBuffer.class));
+    // should not merge the ranges
+    VectoredReadUtils.readVectored(stream, input, ByteBuffer::allocate, 100, 100);
+    Mockito.verify(stream, Mockito.times(3))
+        .readFully(ArgumentMatchers.anyLong(), ArgumentMatchers.any(ByteBuffer.class));
+    for(int b=0; b < input.size(); ++b) {
+      validateBuffer("buffer " + b, input.get(b).getData().get(), 0);
+    }
+  }
+
+  @Test
+  public void testReadVectoredMerge() throws Exception {
+    List<FileRange> input = Arrays.asList(new FileRangeImpl(2000, 100),
+        new FileRangeImpl(1000, 100),
+        new FileRangeImpl(0, 100));
+    Stream stream = Mockito.mock(Stream.class);
+    Mockito.doAnswer(invocation -> {
+      fillBuffer(invocation.getArgument(1));
+      return null;
+    }).when(stream).readFully(ArgumentMatchers.anyLong(),
+        ArgumentMatchers.any(ByteBuffer.class));
+    // should merge the ranges into a single read
+    VectoredReadUtils.readVectored(stream, input, ByteBuffer::allocate, 1000, 2100);
+    Mockito.verify(stream, Mockito.times(1))
+        .readFully(ArgumentMatchers.anyLong(), ArgumentMatchers.any(ByteBuffer.class));
+    for(int b=0; b < input.size(); ++b) {
+      validateBuffer("buffer " + b, input.get(b).getData().get(), (2 - b) * 1000);
+    }
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java
index 142669b7868..f83ef9e63d1 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java
@@ -19,6 +19,9 @@
 package org.apache.hadoop.test;
 
 import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+
+import org.assertj.core.api.Assertions;
 import org.junit.Assert;
 
 /**
@@ -28,17 +31,18 @@ public class MoreAsserts {
 
   /**
    * Assert equivalence for array and iterable
-   * @param <T> the type of the elements
-   * @param s the name/message for the collection
-   * @param expected  the expected array of elements
-   * @param actual    the actual iterable of elements
+   *
+   * @param <T>      the type of the elements
+   * @param s        the name/message for the collection
+   * @param expected the expected array of elements
+   * @param actual   the actual iterable of elements
    */
   public static <T> void assertEquals(String s, T[] expected,
                                       Iterable<T> actual) {
     Iterator<T> it = actual.iterator();
     int i = 0;
     for (; i < expected.length && it.hasNext(); ++i) {
-      Assert.assertEquals("Element "+ i +" for "+ s, expected[i], it.next());
+      Assert.assertEquals("Element " + i + " for " + s, expected[i], it.next());
     }
     Assert.assertTrue("Expected more elements", i == expected.length);
     Assert.assertTrue("Expected less elements", !it.hasNext());
@@ -46,7 +50,8 @@ public class MoreAsserts {
 
   /**
    * Assert equality for two iterables
-   * @param <T> the type of the elements
+   *
+   * @param <T>      the type of the elements
    * @param s
    * @param expected
    * @param actual
@@ -57,10 +62,28 @@ public class MoreAsserts {
     Iterator<T> ita = actual.iterator();
     int i = 0;
     while (ite.hasNext() && ita.hasNext()) {
-      Assert.assertEquals("Element "+ i +" for "+s, ite.next(), ita.next());
+      Assert.assertEquals("Element " + i + " for " + s, ite.next(), ita.next());
     }
     Assert.assertTrue("Expected more elements", !ite.hasNext());
     Assert.assertTrue("Expected less elements", !ita.hasNext());
   }
 
+
+  public static <T> void assertFutureCompletedSuccessfully(CompletableFuture<T> future) {
+    Assertions.assertThat(future.isDone())
+            .describedAs("This future is supposed to be " +
+                    "completed successfully")
+            .isTrue();
+    Assertions.assertThat(future.isCompletedExceptionally())
+            .describedAs("This future is supposed to be " +
+                    "completed successfully")
+            .isFalse();
+  }
+
+  public static <T> void assertFutureFailedExceptionally(CompletableFuture<T> future) {
+    Assertions.assertThat(future.isCompletedExceptionally())
+            .describedAs("This future is supposed to be " +
+                    "completed exceptionally")
+            .isTrue();
+  }
 }
diff --git a/hadoop-common-project/pom.xml b/hadoop-common-project/pom.xml
index 6064b8c494d..b2a5fd3a4fa 100644
--- a/hadoop-common-project/pom.xml
+++ b/hadoop-common-project/pom.xml
@@ -56,5 +56,4 @@
       </plugin>
     </plugins>
   </build>
-
 </project>
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 8dc1862ed57..39dac39eac7 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -218,6 +218,7 @@
     <nodejs.version>v12.22.1</nodejs.version>
     <yarnpkg.version>v1.22.5</yarnpkg.version>
     <apache-ant.version>1.10.11</apache-ant.version>
+    <jmh.version>1.20</jmh.version>
   </properties>
 
   <dependencyManagement>
@@ -1603,6 +1604,16 @@
          </exclusion>
        </exclusions>
      </dependency>
+      <dependency>
+        <groupId>org.openjdk.jmh</groupId>
+        <artifactId>jmh-core</artifactId>
+        <version>${jmh.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.openjdk.jmh</groupId>
+        <artifactId>jmh-generator-annprocess</artifactId>
+        <version>${jmh.version}</version>
+      </dependency>
      <dependency>
        <groupId>org.apache.curator</groupId>
        <artifactId>curator-test</artifactId>
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 149216441f2..943416e2d87 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -1463,11 +1463,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     fileInformation.applyOptions(readContext);
     LOG.debug("Opening '{}'", readContext);
     return new FSDataInputStream(
-        new S3AInputStream(
-            readContext.build(),
-            createObjectAttributes(path, fileStatus),
-            createInputStreamCallbacks(auditSpan),
-            inputStreamStats));
+            new S3AInputStream(
+                  readContext.build(),
+                  createObjectAttributes(path, fileStatus),
+                  createInputStreamCallbacks(auditSpan),
+                  inputStreamStats,
+                  unboundedThreadPool));
   }
 
   /**
@@ -4925,9 +4926,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   /**
    * This is a proof of concept of a select API.
    * @param source path to source data
-   * @param expression select expression
    * @param options request configuration from the builder.
-   * @param providedStatus any passed in status
+   * @param fileInformation any passed in information.
    * @return the stream of the results
    * @throws IOException IO failure
    */
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index 6beeb2891ee..05d9c7f9fe9 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -19,38 +19,49 @@
 package org.apache.hadoop.fs.s3a;
 
 import javax.annotation.Nullable;
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.function.IntFunction;
 
 import com.amazonaws.services.s3.model.GetObjectRequest;
 import com.amazonaws.services.s3.model.S3Object;
 import com.amazonaws.services.s3.model.S3ObjectInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.classification.VisibleForTesting;
-import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.fs.CanSetReadahead;
 import org.apache.hadoop.fs.CanUnbuffer;
 import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.impl.CombinedFileRange;
+import org.apache.hadoop.fs.impl.FutureIOSupport;
+import org.apache.hadoop.fs.impl.VectoredReadUtils;
 import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.statistics.DurationTracker;
 import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
-import org.apache.hadoop.fs.PathIOException;
-import org.apache.hadoop.fs.StreamCapabilities;
-import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.EOFException;
-import java.io.IOException;
-import java.net.SocketTimeoutException;
-import java.util.concurrent.CompletableFuture;
-
 import static java.util.Objects.requireNonNull;
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
+import static org.apache.hadoop.fs.impl.VectoredReadUtils.isOrderedDisjoint;
+import static org.apache.hadoop.fs.impl.VectoredReadUtils.sliceTo;
+import static org.apache.hadoop.fs.impl.VectoredReadUtils.sortAndMergeRanges;
 import static org.apache.hadoop.fs.s3a.Invoker.onceTrackingDuration;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
 import static org.apache.hadoop.util.StringUtils.toLowerCase;
@@ -88,6 +99,13 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * size of a buffer to create when draining the stream.
    */
   private static final int DRAIN_BUFFER_SIZE = 16384;
+  /**
+   * This is the maximum temporary buffer size we use while
+   * populating the data in direct byte buffers during a vectored IO
+   * operation. This is to ensure that when a big range of data is
+   * requested in direct byte buffer doesn't leads to OOM errors.
+   */
+  private static final int TMP_BUFFER_MAX_SIZE = 64 * 1024;
 
   /**
    * This is the public position; the one set in {@link #seek(long)}
@@ -111,6 +129,11 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   private S3ObjectInputStream wrappedStream;
   private final S3AReadOpContext context;
   private final InputStreamCallbacks client;
+
+  /**
+   * Thread pool used for vectored IO operation.
+   */
+  private final ThreadPoolExecutor unboundedThreadPool;
   private final String bucket;
   private final String key;
   private final String pathStr;
@@ -160,12 +183,13 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * @param ctx operation context
    * @param s3Attributes object attributes
    * @param client S3 client to use
-   * @param streamStatistics statistics for this stream
+   * @param unboundedThreadPool thread pool to use.
    */
   public S3AInputStream(S3AReadOpContext ctx,
-      S3ObjectAttributes s3Attributes,
-      InputStreamCallbacks client,
-      S3AInputStreamStatistics streamStatistics) {
+                        S3ObjectAttributes s3Attributes,
+                        InputStreamCallbacks client,
+                        S3AInputStreamStatistics streamStatistics,
+                        ThreadPoolExecutor unboundedThreadPool) {
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()),
         "No Bucket");
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key");
@@ -187,6 +211,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     setInputPolicy(ctx.getInputPolicy());
     setReadahead(ctx.getReadahead());
     this.asyncDrainThreshold = ctx.getAsyncDrainThreshold();
+    this.unboundedThreadPool = unboundedThreadPool;
   }
 
   /**
@@ -880,6 +905,231 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     }
   }
 
+  /**
+   * {@inheritDoc}
+   * Vectored read implementation for S3AInputStream.
+   * @param ranges the byte ranges to read.
+   * @param allocate the function to allocate ByteBuffer.
+   * @throws IOException IOE if any.
+   */
+  @Override
+  public void readVectored(List<? extends FileRange> ranges,
+                           IntFunction<ByteBuffer> allocate) throws IOException {
+
+    LOG.debug("Starting vectored read on path {} for ranges {} ", pathStr, ranges);
+    checkNotClosed();
+    for (FileRange range : ranges) {
+      validateRangeRequest(range);
+      CompletableFuture<ByteBuffer> result = new CompletableFuture<>();
+      range.setData(result);
+    }
+
+    if (isOrderedDisjoint(ranges, 1, minSeekForVectorReads())) {
+      LOG.debug("Not merging the ranges as they are disjoint");
+      for(FileRange range: ranges) {
+        ByteBuffer buffer = allocate.apply(range.getLength());
+        unboundedThreadPool.submit(() -> readSingleRange(range, buffer));
+      }
+    } else {
+      LOG.debug("Trying to merge the ranges as they are not disjoint");
+      List<CombinedFileRange> combinedFileRanges = sortAndMergeRanges(ranges,
+              1, minSeekForVectorReads(),
+              maxReadSizeForVectorReads());
+      LOG.debug("Number of original ranges size {} , Number of combined ranges {} ",
+              ranges.size(), combinedFileRanges.size());
+      for(CombinedFileRange combinedFileRange: combinedFileRanges) {
+        CompletableFuture<ByteBuffer> result = new CompletableFuture<>();
+        ByteBuffer buffer = allocate.apply(combinedFileRange.getLength());
+        combinedFileRange.setData(result);
+        unboundedThreadPool.submit(
+            () -> readCombinedRangeAndUpdateChildren(combinedFileRange, buffer));
+      }
+    }
+    LOG.debug("Finished submitting vectored read to threadpool" +
+            " on path {} for ranges {} ", pathStr, ranges);
+  }
+
+  /**
+   * Read data in the combinedFileRange and update data in buffers
+   * of all underlying ranges.
+   * @param combinedFileRange combined range.
+   * @param buffer combined buffer.
+   */
+  private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRange,
+                                                  ByteBuffer buffer) {
+    // Not putting read single range call inside try block as
+    // exception if any occurred during this call will be raised
+    // during awaitFuture call while getting the combined buffer.
+    readSingleRange(combinedFileRange, buffer);
+    try {
+      // In case of single range we return the original byte buffer else
+      // we return slice byte buffers for each child ranges.
+      ByteBuffer combinedBuffer = FutureIOSupport.awaitFuture(combinedFileRange.getData());
+      if (combinedFileRange.getUnderlying().size() == 1) {
+        combinedFileRange.getUnderlying().get(0).getData().complete(combinedBuffer);
+      } else {
+        for (FileRange child : combinedFileRange.getUnderlying()) {
+          updateOriginalRange(child, combinedBuffer, combinedFileRange);
+        }
+      }
+    } catch (Exception ex) {
+      LOG.warn("Exception occurred while reading combined range from file {}", pathStr, ex);
+      for(FileRange child : combinedFileRange.getUnderlying()) {
+        child.getData().completeExceptionally(ex);
+      }
+    }
+  }
+
+  /**
+   * Update data in child range from combined range.
+   * @param child child range.
+   * @param combinedBuffer combined buffer.
+   * @param combinedFileRange combined range.
+   */
+  private void updateOriginalRange(FileRange child,
+                                   ByteBuffer combinedBuffer,
+                                   CombinedFileRange combinedFileRange) {
+    LOG.trace("Start Filling original range [{}, {}) from combined range [{}, {}) ",
+            child.getOffset(), child.getLength(),
+            combinedFileRange.getOffset(), combinedFileRange.getLength());
+    ByteBuffer childBuffer = sliceTo(combinedBuffer, combinedFileRange.getOffset(), child);
+    child.getData().complete(childBuffer);
+    LOG.trace("End Filling original range [{}, {}) from combined range [{}, {}) ",
+            child.getOffset(), child.getLength(),
+            combinedFileRange.getOffset(), combinedFileRange.getLength());
+  }
+
+  /**
+   *  // Check if we can use contentLength returned by http GET request.
+   * Validates range parameters.
+   * @param range requested range.
+   * @throws EOFException end of file exception.
+   */
+  private void validateRangeRequest(FileRange range) throws EOFException {
+    VectoredReadUtils.validateRangeRequest(range);
+    if(range.getOffset() + range.getLength() > contentLength) {
+      LOG.warn("Requested range [{}, {}) is beyond EOF for path {}",
+              range.getOffset(), range.getLength(), pathStr);
+      throw new EOFException("Requested range [" + range.getOffset() +", "
+              + range.getLength() + ") is beyond EOF for path " + pathStr);
+    }
+  }
+
+  /**
+   * TODO: Add retry in client.getObject(). not present in older reads why here??
+   * Okay retry is being done in the top layer during read.
+   * But if we do here in the top layer, one issue I am thinking is
+   * what if there is some error which happened during filling the buffer
+   * If we retry that old offsets of heap buffers can be overwritten ?
+   * I think retry should be only added in {@link S3AInputStream#getS3Object}
+   * Read data from S3 for this range and populate the bufffer.
+   * @param range range of data to read.
+   * @param buffer buffer to fill.
+   */
+  private void readSingleRange(FileRange range, ByteBuffer buffer) {
+    LOG.debug("Start reading range {} from path {} ", range, pathStr);
+    S3Object objectRange = null;
+    S3ObjectInputStream objectContent = null;
+    try {
+      long position = range.getOffset();
+      int length = range.getLength();
+      final String operationName = "readRange";
+      objectRange = getS3Object(operationName, position, length);
+      objectContent = objectRange.getObjectContent();
+      if (objectContent == null) {
+        throw new PathIOException(uri,
+                "Null IO stream received during " + operationName);
+      }
+      populateBuffer(length, buffer, objectContent);
+      range.getData().complete(buffer);
+    } catch (Exception ex) {
+      LOG.warn("Exception while reading a range {} from path {} ", range, pathStr, ex);
+      range.getData().completeExceptionally(ex);
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, objectRange, objectContent);
+    }
+    LOG.debug("Finished reading range {} from path {} ", range, pathStr);
+  }
+
+  /**
+   * Populates the buffer with data from objectContent
+   * till length. Handles both direct and heap byte buffers.
+   * @param length length of data to populate.
+   * @param buffer buffer to fill.
+   * @param objectContent result retrieved from S3 store.
+   * @throws IOException any IOE.
+   */
+  private void populateBuffer(int length,
+                              ByteBuffer buffer,
+                              S3ObjectInputStream objectContent) throws IOException {
+    if (buffer.isDirect()) {
+      int readBytes = 0;
+      int offset = 0;
+      byte[] tmp = new byte[TMP_BUFFER_MAX_SIZE];
+      while (readBytes < length) {
+        int currentLength = readBytes + TMP_BUFFER_MAX_SIZE < length ?
+                TMP_BUFFER_MAX_SIZE
+                : length - readBytes;
+        readByteArray(objectContent, tmp, 0, currentLength);
+        buffer.put(tmp, 0, currentLength);
+        offset = offset + currentLength;
+        readBytes = readBytes + currentLength;
+      }
+      buffer.flip();
+    } else {
+      readByteArray(objectContent, buffer.array(), 0, length);
+    }
+  }
+
+  public void readByteArray(S3ObjectInputStream objectContent,
+                            byte[] dest,
+                            int offset,
+                            int length) throws IOException {
+    int readBytes = 0;
+    while (readBytes < length) {
+      int readBytesCurr = objectContent.read(dest,
+              offset + readBytes,
+              length - readBytes);
+      readBytes +=readBytesCurr;
+      if (readBytesCurr < 0) {
+        throw new EOFException(FSExceptionMessages.EOF_IN_READ_FULLY);
+      }
+    }
+  }
+
+  /**
+   * Read data from S3 using a http request.
+   * This also handles if file has been changed while http call
+   * is getting executed. If file has been changed RemoteFileChangedException
+   * is thrown.
+   * @param operationName name of the operation for which get object on S3 is called.
+   * @param position position of the object to be read from S3.
+   * @param length length from position of the object to be read from S3.
+   * @return S3Object
+   * @throws IOException exception if any.
+   */
+  private S3Object getS3Object(String operationName, long position,
+                               int length) throws IOException {
+    final GetObjectRequest request = client.newGetRequest(key)
+            .withRange(position, position + length - 1);
+    changeTracker.maybeApplyConstraint(request);
+    DurationTracker tracker = streamStatistics.initiateGetRequest();
+    S3Object objectRange;
+    Invoker invoker = context.getReadInvoker();
+    try {
+      objectRange = invoker.retry(operationName, pathStr, true,
+          () -> client.getObject(request));
+    } catch (IOException ex) {
+      tracker.failed();
+      throw ex;
+    } finally {
+      tracker.close();
+    }
+    changeTracker.processResponse(objectRange, operationName,
+            position);
+    return objectRange;
+  }
+
   /**
    * Access the input stream statistics.
    * This is for internal testing and may be removed without warning.
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java
new file mode 100644
index 00000000000..255cc6501c2
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.FileRangeImpl;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTest {
+
+  public ITestS3AContractVectoredRead(String bufferType) {
+    super(bufferType);
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new S3AContract(conf);
+  }
+
+  /**
+   * Overriding in S3 vectored read api fails fast in case of EOF
+   * requested range.
+   * @throws Exception
+   */
+  @Override
+  public void testEOFRanges() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(DATASET_LEN, 100));
+    testExceptionalVectoredRead(fs, fileRanges, "EOFException is expected");
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
index 62f5bff35c4..c62bf5daca3 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
@@ -111,7 +111,8 @@ public class TestS3AInputStreamRetry extends AbstractS3AMockTest {
         s3AReadOpContext,
         s3ObjectAttributes,
         getMockedInputStreamCallback(),
-        s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics());
+        s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(),
+            null);
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
index fc287e9845c..c831999008f 100644
--- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
@@ -52,7 +52,7 @@ log4j.logger.org.apache.hadoop.ipc.Server=WARN
 
 # for debugging low level S3a operations, uncomment these lines
 # Log all S3A classes
-#log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
+log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
 #log4j.logger.org.apache.hadoop.fs.s3a.S3AUtils=INFO
 #log4j.logger.org.apache.hadoop.fs.s3a.Listing=INFO
 
diff --git a/hadoop-tools/hadoop-benchmark/pom.xml b/hadoop-tools/hadoop-benchmark/pom.xml
new file mode 100644
index 00000000000..3d742fab5c8
--- /dev/null
+++ b/hadoop-tools/hadoop-benchmark/pom.xml
@@ -0,0 +1,94 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+                      https://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.4.0-SNAPSHOT</version>
+    <relativePath>../../hadoop-project/pom.xml</relativePath>
+  </parent>
+  <artifactId>hadoop-benchmark</artifactId>
+  <version>3.4.0-SNAPSHOT</version>
+  <packaging>jar</packaging>
+
+  <name>Apache Hadoop Common Benchmark</name>
+  <description>Apache Hadoop Common Benchmark</description>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.openjdk.jmh</groupId>
+      <artifactId>jmh-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.openjdk.jmh</groupId>
+      <artifactId>jmh-generator-annprocess</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-assembly-plugin</artifactId>
+        <configuration>
+          <archive>
+            <manifest>
+              <mainClass>org.apache.hadoop.benchmark.VectoredReadBenchmark</mainClass>
+            </manifest>
+          </archive>
+          <descriptors>
+            <descriptor>src/main/assembly/uber.xml</descriptor>
+          </descriptors>
+        </configuration>
+        <executions>
+          <execution>
+            <id>make-assembly</id> <!-- this is used for inheritance merges -->
+            <phase>package</phase> <!-- bind to the packaging phase -->
+            <goals>
+              <goal>single</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+          <configuration>
+            <excludeFilterFile>${basedir}/src/main/findbugs/exclude.xml</excludeFilterFile>
+          </configuration>
+      </plugin>
+      <plugin>
+        <groupId>com.github.spotbugs</groupId>
+        <artifactId>spotbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/src/main/findbugs/exclude.xml</excludeFilterFile>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/hadoop-tools/hadoop-benchmark/src/main/assembly/uber.xml b/hadoop-tools/hadoop-benchmark/src/main/assembly/uber.xml
new file mode 100644
index 00000000000..014eab951b3
--- /dev/null
+++ b/hadoop-tools/hadoop-benchmark/src/main/assembly/uber.xml
@@ -0,0 +1,33 @@
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<assembly>
+  <id>uber</id>
+  <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <dependencySets>
+    <dependencySet>
+      <outputDirectory>/</outputDirectory>
+      <useProjectArtifact>true</useProjectArtifact>
+      <unpack>true</unpack>
+      <scope>runtime</scope>
+    </dependencySet>
+  </dependencySets>
+  <containerDescriptorHandlers>
+    <containerDescriptorHandler>
+      <handlerName>metaInf-services</handlerName>
+    </containerDescriptorHandler>
+  </containerDescriptorHandlers>
+</assembly>
diff --git a/hadoop-tools/hadoop-benchmark/src/main/findbugs/exclude.xml b/hadoop-tools/hadoop-benchmark/src/main/findbugs/exclude.xml
new file mode 100644
index 00000000000..05f2a067cf0
--- /dev/null
+++ b/hadoop-tools/hadoop-benchmark/src/main/findbugs/exclude.xml
@@ -0,0 +1,22 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<FindBugsFilter>
+  <Match>
+    <Class name="~org\.apache\.hadoop\.benchmark\.generated.*"/>
+  </Match>
+  <Match>
+    <Class name="~org\.openjdk\.jmh\.infra\.generated.*"/>
+  </Match>
+</FindBugsFilter>
diff --git a/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java b/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java
new file mode 100644
index 00000000000..aaee951d72c
--- /dev/null
+++ b/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.benchmark;
+
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousFileChannel;
+import java.nio.channels.CompletionHandler;
+import java.nio.file.FileSystems;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntFunction;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.FileRangeImpl;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MICROSECONDS)
+public class VectoredReadBenchmark {
+
+  static final Path DATA_PATH = getTestDataPath();
+  static final String DATA_PATH_PROPERTY = "bench.data";
+  static final int READ_SIZE = 64 * 1024;
+  static final long SEEK_SIZE = 1024L * 1024;
+
+
+  static Path getTestDataPath() {
+    String value = System.getProperty(DATA_PATH_PROPERTY);
+    return new Path(value == null ? "/tmp/taxi.orc" : value);
+  }
+
+  @State(Scope.Thread)
+  public static class FileSystemChoice {
+
+    @Param({"local", "raw"})
+    private String fileSystemKind;
+
+    private Configuration conf;
+    private FileSystem fs;
+
+    @Setup(Level.Trial)
+    public void setup() {
+      conf = new Configuration();
+      try {
+        LocalFileSystem local = FileSystem.getLocal(conf);
+        fs = "raw".equals(fileSystemKind) ? local.getRaw() : local;
+      } catch (IOException e) {
+        throw new IllegalArgumentException("Can't get filesystem", e);
+      }
+    }
+  }
+
+  @State(Scope.Thread)
+  public static class BufferChoice {
+    @Param({"direct", "array"})
+    private String bufferKind;
+
+    private IntFunction<ByteBuffer> allocate;
+    @Setup(Level.Trial)
+    public void setup() {
+      allocate = "array".equals(bufferKind)
+                     ? ByteBuffer::allocate : ByteBuffer::allocateDirect;
+    }
+  }
+
+  @Benchmark
+  public void asyncRead(FileSystemChoice fsChoice,
+                        BufferChoice bufferChoice,
+                        Blackhole blackhole) throws Exception {
+    FSDataInputStream stream = fsChoice.fs.open(DATA_PATH);
+    List<FileRange> ranges = new ArrayList<>();
+    for(int m=0; m < 100; ++m) {
+      FileRangeImpl range = new FileRangeImpl(m * SEEK_SIZE, READ_SIZE);
+      ranges.add(range);
+    }
+    stream.readVectored(ranges, bufferChoice.allocate);
+    for(FileRange range: ranges) {
+      blackhole.consume(range.getData().get());
+    }
+    stream.close();
+  }
+
+  static class Joiner implements CompletionHandler<ByteBuffer, FileRange> {
+    private int remaining;
+    private final ByteBuffer[] result;
+    private Throwable exception = null;
+
+    Joiner(int total) {
+      remaining = total;
+      result = new ByteBuffer[total];
+    }
+
+    synchronized void finish() {
+      remaining -= 1;
+      if (remaining == 0) {
+        notify();
+      }
+    }
+
+    synchronized ByteBuffer[] join() throws InterruptedException, IOException {
+      while (remaining > 0 && exception == null) {
+        wait();
+      }
+      if (exception != null) {
+        throw new IOException("problem reading", exception);
+      }
+      return result;
+    }
+
+
+    @Override
+    public synchronized void completed(ByteBuffer buffer, FileRange attachment) {
+      result[--remaining] = buffer;
+      if (remaining == 0) {
+        notify();
+      }
+    }
+
+    @Override
+    public synchronized void failed(Throwable exc, FileRange attachment) {
+      this.exception = exc;
+      notify();
+    }
+  }
+
+  static class FileRangeCallback extends FileRangeImpl implements
+          CompletionHandler<Integer, FileRangeCallback> {
+    private final AsynchronousFileChannel channel;
+    private final ByteBuffer buffer;
+    private int completed = 0;
+    private final Joiner joiner;
+
+    FileRangeCallback(AsynchronousFileChannel channel, long offset,
+                      int length, Joiner joiner, ByteBuffer buffer) {
+      super(offset, length);
+      this.channel = channel;
+      this.joiner = joiner;
+      this.buffer = buffer;
+    }
+
+    @Override
+    public void completed(Integer result, FileRangeCallback attachment) {
+      final int bytes = result;
+      if (bytes == -1) {
+        failed(new EOFException("Read past end of file"), this);
+      }
+      completed += bytes;
+      if (completed < this.getLength()) {
+        channel.read(buffer, this.getOffset() + completed, this, this);
+      } else {
+        buffer.flip();
+        joiner.finish();
+      }
+    }
+
+    @Override
+    public void failed(Throwable exc, FileRangeCallback attachment) {
+      joiner.failed(exc, this);
+    }
+  }
+
+  @Benchmark
+  public void asyncFileChanArray(BufferChoice bufferChoice,
+                                 Blackhole blackhole) throws Exception {
+    java.nio.file.Path path = FileSystems.getDefault().getPath(DATA_PATH.toString());
+    AsynchronousFileChannel channel = AsynchronousFileChannel.open(path, StandardOpenOption.READ);
+    List<FileRangeImpl> ranges = new ArrayList<>();
+    Joiner joiner = new Joiner(100);
+    for(int m=0; m < 100; ++m) {
+      ByteBuffer buffer = bufferChoice.allocate.apply(READ_SIZE);
+      FileRangeCallback range = new FileRangeCallback(channel, m * SEEK_SIZE,
+          READ_SIZE, joiner, buffer);
+      ranges.add(range);
+      channel.read(buffer, range.getOffset(), range, range);
+    }
+    joiner.join();
+    channel.close();
+    blackhole.consume(ranges);
+  }
+
+  @Benchmark
+  public void syncRead(FileSystemChoice fsChoice,
+                       Blackhole blackhole) throws Exception {
+    FSDataInputStream stream = fsChoice.fs.open(DATA_PATH);
+    List<byte[]> result = new ArrayList<>();
+    for(int m=0; m < 100; ++m) {
+      byte[] buffer = new byte[READ_SIZE];
+      stream.readFully(m * SEEK_SIZE, buffer);
+      result.add(buffer);
+    }
+    blackhole.consume(result);
+    stream.close();
+  }
+
+  /**
+   * Run the benchmarks.
+   * @param args the pathname of a 100MB data file
+   * @throws Exception any ex.
+   */
+  public static void main(String[] args) throws Exception {
+    OptionsBuilder opts = new OptionsBuilder();
+    opts.include("VectoredReadBenchmark");
+    opts.jvmArgs("-server", "-Xms256m", "-Xmx2g",
+        "-D" + DATA_PATH_PROPERTY + "=" + args[0]);
+    opts.forks(1);
+    new Runner(opts.build()).run();
+  }
+}
diff --git a/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/package-info.java b/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/package-info.java
new file mode 100644
index 00000000000..95d6977e3ab
--- /dev/null
+++ b/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Benchmark for Vectored Read IO operations.
+ */
+package org.apache.hadoop.benchmark;
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index 860e71a8ea6..b05e489261f 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -50,6 +50,7 @@
     <module>hadoop-azure-datalake</module>
     <module>hadoop-aliyun</module>
     <module>hadoop-fs2img</module>
+    <module>hadoop-benchmark</module>
   </modules>
 
   <build>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[hadoop] 05/05: HADOOP-18106: Handle memory fragmentation in S3A Vectored IO. (#4445)

Posted by mt...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mthakur pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit c517b086f2c31c3b3891f53699642950575a8716
Author: Mukund Thakur <mt...@cloudera.com>
AuthorDate: Tue Jun 21 03:45:40 2022 +0530

    HADOOP-18106: Handle memory fragmentation in S3A Vectored IO. (#4445)
    
    part of HADOOP-18103.
    Handling memory fragmentation in S3A vectored IO implementation by
    allocating smaller user range requested size buffers and directly
    filling them from the remote S3 stream and skipping undesired
    data in between ranges.
    This patch also adds aborting active vectored reads when stream is
    closed or unbuffer() is called.
    
    Contributed By: Mukund Thakur
    
     Conflicts:
            hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
---
 .../org/apache/hadoop/fs/ChecksumFileSystem.java   |  11 +-
 .../main/java/org/apache/hadoop/fs/FileRange.java  |  12 ++
 .../org/apache/hadoop/fs/PositionedReadable.java   |   4 +-
 .../org/apache/hadoop/fs/RawLocalFileSystem.java   |  16 +-
 .../org/apache/hadoop/fs/StreamCapabilities.java   |   6 +
 .../hadoop/fs/{impl => }/VectoredReadUtils.java    |  91 +++++----
 .../apache/hadoop/fs/impl/CombinedFileRange.java   |   1 -
 .../apache/hadoop/fs/{ => impl}/FileRangeImpl.java |   9 +-
 .../site/markdown/filesystem/fsdatainputstream.md  |  11 +-
 .../fs/{impl => }/TestVectoredReadUtils.java       | 149 +++++++--------
 .../contract/AbstractContractVectoredReadTest.java | 198 ++++++++++++++------
 .../hadoop/fs/contract/ContractTestUtils.java      |  20 ++
 .../localfs/TestLocalFSContractVectoredRead.java   |  51 +++++
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   | 205 ++++++++++++++-------
 .../contract/s3a/ITestS3AContractVectoredRead.java |  63 ++++++-
 .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java    |  15 +-
 .../hadoop/benchmark/VectoredReadBenchmark.java    |   4 +-
 17 files changed, 609 insertions(+), 257 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
index e612713cfed..b23df1713c0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
-import org.apache.hadoop.fs.impl.VectoredReadUtils;
 import org.apache.hadoop.fs.impl.CombinedFileRange;
 import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl;
 import org.apache.hadoop.fs.impl.OpenFileParameters;
@@ -55,6 +54,7 @@ import org.apache.hadoop.util.Progressable;
 import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS;
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
 import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable;
+import static org.apache.hadoop.fs.VectoredReadUtils.sortRanges;
 
 /****************************************************************
  * Abstract Checksumed FileSystem.
@@ -166,7 +166,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
    * It verifies that data matches checksums.
    *******************************************************/
   private static class ChecksumFSInputChecker extends FSInputChecker implements
-      IOStatisticsSource {
+      IOStatisticsSource, StreamCapabilities {
     private ChecksumFileSystem fs;
     private FSDataInputStream datas;
     private FSDataInputStream sums;
@@ -408,7 +408,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       int minSeek = minSeekForVectorReads();
       int maxSize = maxReadSizeForVectorReads();
       List<CombinedFileRange> dataRanges =
-          VectoredReadUtils.sortAndMergeRanges(ranges, bytesPerSum,
+          VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(ranges)), bytesPerSum,
               minSeek, maxReadSizeForVectorReads());
       List<CombinedFileRange> checksumRanges = findChecksumRanges(dataRanges,
           bytesPerSum, minSeek, maxSize);
@@ -435,6 +435,11 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
         }
       }
     }
+
+    @Override
+    public boolean hasCapability(String capability) {
+      return datas.hasCapability(capability);
+    }
   }
 
   private static class FSDataBoundedInputStream extends FSDataInputStream {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java
index 7388e462cc2..e55696e9650 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.fs;
 import java.nio.ByteBuffer;
 import java.util.concurrent.CompletableFuture;
 
+import org.apache.hadoop.fs.impl.FileRangeImpl;
+
 /**
  * A byte range of a file.
  * This is used for the asynchronous gather read API of
@@ -52,4 +54,14 @@ public interface FileRange {
    * @param data the future of the ByteBuffer that will have the data
    */
   void setData(CompletableFuture<ByteBuffer> data);
+
+  /**
+   * Factory method to create a FileRange object.
+   * @param offset starting offset of the range.
+   * @param length length of the range.
+   * @return a new instance of FileRangeImpl.
+   */
+  static FileRange createFileRange(long offset, int length) {
+    return new FileRangeImpl(offset, length);
+  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PositionedReadable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PositionedReadable.java
index 7e543ebf226..de760905127 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PositionedReadable.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PositionedReadable.java
@@ -25,7 +25,6 @@ import java.util.function.IntFunction;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.impl.VectoredReadUtils;
 
 /**
  * Stream that permits positional reading.
@@ -121,7 +120,6 @@ public interface PositionedReadable {
    */
   default void readVectored(List<? extends FileRange> ranges,
                             IntFunction<ByteBuffer> allocate) throws IOException {
-    VectoredReadUtils.readVectored(this, ranges, allocate,  minSeekForVectorReads(),
-        maxReadSizeForVectorReads());
+    VectoredReadUtils.readVectored(this, ranges, allocate);
   }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index c71692e67e3..e3baf8cd5be 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -20,7 +20,6 @@
 package org.apache.hadoop.fs;
 
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.fs.impl.VectoredReadUtils;
 
 import java.io.BufferedOutputStream;
 import java.io.DataOutput;
@@ -68,6 +67,7 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
+import static org.apache.hadoop.fs.VectoredReadUtils.sortRanges;
 import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
 import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_EXCEPTIONS;
 import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS;
@@ -278,6 +278,7 @@ public class RawLocalFileSystem extends FileSystem {
       // new capabilities.
       switch (capability.toLowerCase(Locale.ENGLISH)) {
       case StreamCapabilities.IOSTATISTICS:
+      case StreamCapabilities.VECTOREDIO:
         return true;
       default:
         return false;
@@ -303,23 +304,24 @@ public class RawLocalFileSystem extends FileSystem {
     public void readVectored(List<? extends FileRange> ranges,
                              IntFunction<ByteBuffer> allocate) throws IOException {
 
+      List<? extends FileRange> sortedRanges = Arrays.asList(sortRanges(ranges));
       // Set up all of the futures, so that we can use them if things fail
-      for(FileRange range: ranges) {
+      for(FileRange range: sortedRanges) {
         VectoredReadUtils.validateRangeRequest(range);
         range.setData(new CompletableFuture<>());
       }
       try {
         AsynchronousFileChannel channel = getAsyncChannel();
-        ByteBuffer[] buffers = new ByteBuffer[ranges.size()];
-        AsyncHandler asyncHandler = new AsyncHandler(channel, ranges, buffers);
-        for(int i = 0; i < ranges.size(); ++i) {
-          FileRange range = ranges.get(i);
+        ByteBuffer[] buffers = new ByteBuffer[sortedRanges.size()];
+        AsyncHandler asyncHandler = new AsyncHandler(channel, sortedRanges, buffers);
+        for(int i = 0; i < sortedRanges.size(); ++i) {
+          FileRange range = sortedRanges.get(i);
           buffers[i] = allocate.apply(range.getLength());
           channel.read(buffers[i], range.getOffset(), i, asyncHandler);
         }
       } catch (IOException ioe) {
         LOG.debug("Exception occurred during vectored read ", ioe);
-        for(FileRange range: ranges) {
+        for(FileRange range: sortedRanges) {
           range.getData().completeExceptionally(ioe);
         }
       }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java
index 86117801950..d68ef505dc3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java
@@ -80,6 +80,12 @@ public interface StreamCapabilities {
    */
   String IOSTATISTICS = "iostatistics";
 
+  /**
+   * Support for vectored IO api.
+   * See {@code PositionedReadable#readVectored(List, IntFunction)}.
+   */
+  String VECTOREDIO = "readvectored";
+
   /**
    * Stream abort() capability implemented by {@link Abortable#abort()}.
    * This matches the Path Capability
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/VectoredReadUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java
similarity index 79%
rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/VectoredReadUtils.java
rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java
index 9a16e6841dd..64107f1a18f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/VectoredReadUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.impl;
+package org.apache.hadoop.fs;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -28,9 +28,7 @@ import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.IntFunction;
 
-import org.apache.hadoop.fs.ByteBufferPositionedReadable;
-import org.apache.hadoop.fs.FileRange;
-import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.impl.CombinedFileRange;
 import org.apache.hadoop.util.Preconditions;
 
 /**
@@ -68,35 +66,19 @@ public final class VectoredReadUtils {
 
 
   /**
-   * Read fully a list of file ranges asynchronously from this file.
-   * The default iterates through the ranges to read each synchronously, but
-   * the intent is that subclasses can make more efficient readers.
+   * This is the default implementation which iterates through the ranges
+   * to read each synchronously, but the intent is that subclasses
+   * can make more efficient readers.
    * The data or exceptions are pushed into {@link FileRange#getData()}.
    * @param stream the stream to read the data from
    * @param ranges the byte ranges to read
    * @param allocate the byte buffer allocation
-   * @param minimumSeek the minimum number of bytes to seek over
-   * @param maximumRead the largest number of bytes to combine into a single read
    */
   public static void readVectored(PositionedReadable stream,
                                   List<? extends FileRange> ranges,
-                                  IntFunction<ByteBuffer> allocate,
-                                  int minimumSeek,
-                                  int maximumRead) {
-    if (isOrderedDisjoint(ranges, 1, minimumSeek)) {
-      for(FileRange range: ranges) {
-        range.setData(readRangeFrom(stream, range, allocate));
-      }
-    } else {
-      for(CombinedFileRange range: sortAndMergeRanges(ranges, 1, minimumSeek,
-          maximumRead)) {
-        CompletableFuture<ByteBuffer> read =
-            readRangeFrom(stream, range, allocate);
-        for(FileRange child: range.getUnderlying()) {
-          child.setData(read.thenApply(
-              (b) -> sliceTo(b, range.getOffset(), child)));
-        }
-      }
+                                  IntFunction<ByteBuffer> allocate) {
+    for (FileRange range: ranges) {
+      range.setData(readRangeFrom(stream, range, allocate));
     }
   }
 
@@ -166,7 +148,7 @@ public final class VectoredReadUtils {
                                           int chunkSize,
                                           int minimumSeek) {
     long previous = -minimumSeek;
-    for(FileRange range: input) {
+    for (FileRange range: input) {
       long offset = range.getOffset();
       long end = range.getOffset() + range.getLength();
       if (offset % chunkSize != 0 ||
@@ -209,7 +191,42 @@ public final class VectoredReadUtils {
   }
 
   /**
-   * Sort and merge ranges to optimize the access from the underlying file
+   * Check if the input ranges are overlapping in nature.
+   * We call two ranges to be overlapping when start offset
+   * of second is less than the end offset of first.
+   * End offset is calculated as start offset + length.
+   * @param input list if input ranges.
+   * @return true/false based on logic explained above.
+   */
+  public static List<? extends FileRange> validateNonOverlappingAndReturnSortedRanges(
+          List<? extends FileRange> input) {
+
+    if (input.size() <= 1) {
+      return input;
+    }
+    FileRange[] sortedRanges = sortRanges(input);
+    FileRange prev = sortedRanges[0];
+    for (int i=1; i<sortedRanges.length; i++) {
+      if (sortedRanges[i].getOffset() < prev.getOffset() + prev.getLength()) {
+        throw new UnsupportedOperationException("Overlapping ranges are not supported");
+      }
+    }
+    return Arrays.asList(sortedRanges);
+  }
+
+  /**
+   * Sort the input ranges by offset.
+   * @param input input ranges.
+   * @return sorted ranges.
+   */
+  public static FileRange[] sortRanges(List<? extends FileRange> input) {
+    FileRange[] sortedRanges = input.toArray(new FileRange[0]);
+    Arrays.sort(sortedRanges, Comparator.comparingLong(FileRange::getOffset));
+    return sortedRanges;
+  }
+
+  /**
+   * Merge sorted ranges to optimize the access from the underlying file
    * system.
    * The motivations are that:
    * <ul>
@@ -219,24 +236,22 @@ public final class VectoredReadUtils {
    *   <li>Some file systems want to round ranges to be at checksum boundaries.</li>
    * </ul>
    *
-   * @param input the list of input ranges
+   * @param sortedRanges already sorted list of ranges based on offset.
    * @param chunkSize round the start and end points to multiples of chunkSize
    * @param minimumSeek the smallest gap that we should seek over in bytes
    * @param maxSize the largest combined file range in bytes
    * @return the list of sorted CombinedFileRanges that cover the input
    */
-  public static List<CombinedFileRange> sortAndMergeRanges(List<? extends FileRange> input,
-                                                           int chunkSize,
-                                                           int minimumSeek,
-                                                           int maxSize) {
-    // sort the ranges by offset
-    FileRange[] ranges = input.toArray(new FileRange[0]);
-    Arrays.sort(ranges, Comparator.comparingLong(FileRange::getOffset));
+  public static List<CombinedFileRange> mergeSortedRanges(List<? extends FileRange> sortedRanges,
+                                                          int chunkSize,
+                                                          int minimumSeek,
+                                                          int maxSize) {
+
     CombinedFileRange current = null;
-    List<CombinedFileRange> result = new ArrayList<>(ranges.length);
+    List<CombinedFileRange> result = new ArrayList<>(sortedRanges.size());
 
     // now merge together the ones that merge
-    for(FileRange range: ranges) {
+    for (FileRange range: sortedRanges) {
       long start = roundDown(range.getOffset(), chunkSize);
       long end = roundUp(range.getOffset() + range.getLength(), chunkSize);
       if (current == null || !current.merge(start, end, range, minimumSeek, maxSize)) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java
index 828a50b4f7b..516bbb2c70c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.fs.impl;
 
 import org.apache.hadoop.fs.FileRange;
-import org.apache.hadoop.fs.FileRangeImpl;
 
 import java.util.ArrayList;
 import java.util.List;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRangeImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileRangeImpl.java
similarity index 85%
rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRangeImpl.java
rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileRangeImpl.java
index ef5851154be..041e5f0a8d2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRangeImpl.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileRangeImpl.java
@@ -15,15 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.fs;
+package org.apache.hadoop.fs.impl;
 
 import java.nio.ByteBuffer;
 import java.util.concurrent.CompletableFuture;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileRange;
+
 /**
  * A range of bytes from a file with an optional buffer to read those bytes
- * for zero copy.
+ * for zero copy. This shouldn't be created directly via constructor rather
+ * factory defined in {@code FileRange#createFileRange} should be used.
  */
+@InterfaceAudience.Private
 public class FileRangeImpl implements FileRange {
   private long offset;
   private int length;
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
index e4a2830967e..197b999c81f 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
@@ -449,7 +449,14 @@ Read fully data for a list of ranges asynchronously. The default implementation
 iterates through the ranges, tries to coalesce the ranges based on values of
 `minSeekForVectorReads` and `maxReadSizeForVectorReads` and then read each merged
 ranges synchronously, but the intent is sub classes can implement efficient
-implementation.
+implementation. Reading in both direct and heap byte buffers are supported.
+Also, clients are encouraged to use `WeakReferencedElasticByteBufferPool` for
+allocating buffers such that even direct buffers are garbage collected when
+they are no longer referenced.
+
+Note: Don't use direct buffers for reading from ChecksumFileSystem as that may
+lead to memory fragmentation explained in HADOOP-18296.
+
 
 #### Preconditions
 
@@ -467,7 +474,7 @@ For each requested range:
 
 ### `minSeekForVectorReads()`
 
-Smallest reasonable seek. Two ranges won't be merged together if the difference between
+The smallest reasonable seek. Two ranges won't be merged together if the difference between
 end of first and start of next range is more than this value.
 
 ### `maxReadSizeForVectorReads()`
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java
similarity index 74%
rename from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java
rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java
index cfd366701be..5d08b02e113 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.impl;
+package org.apache.hadoop.fs;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -31,12 +31,10 @@ import org.junit.Test;
 import org.mockito.ArgumentMatchers;
 import org.mockito.Mockito;
 
-import org.apache.hadoop.fs.ByteBufferPositionedReadable;
-import org.apache.hadoop.fs.FileRange;
-import org.apache.hadoop.fs.FileRangeImpl;
-import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.impl.CombinedFileRange;
 import org.apache.hadoop.test.HadoopTestBase;
 
+import static org.apache.hadoop.fs.VectoredReadUtils.sortRanges;
 import static org.apache.hadoop.test.MoreAsserts.assertFutureCompletedSuccessfully;
 import static org.apache.hadoop.test.MoreAsserts.assertFutureFailedExceptionally;
 
@@ -56,7 +54,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
     }
     // ensure we don't make unnecessary slices
     ByteBuffer slice = VectoredReadUtils.sliceTo(buffer, 100,
-        new FileRangeImpl(100, size));
+        FileRange.createFileRange(100, size));
     Assertions.assertThat(buffer)
             .describedAs("Slicing on the same offset shouldn't " +
                     "create a new buffer")
@@ -67,7 +65,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
     final int sliceStart = 1024;
     final int sliceLength = 16 * 1024;
     slice = VectoredReadUtils.sliceTo(buffer, offset,
-        new FileRangeImpl(offset + sliceStart, sliceLength));
+        FileRange.createFileRange(offset + sliceStart, sliceLength));
     // make sure they aren't the same, but use the same backing data
     Assertions.assertThat(buffer)
             .describedAs("Slicing on new offset should " +
@@ -96,12 +94,12 @@ public class TestVectoredReadUtils extends HadoopTestBase {
 
   @Test
   public void testMerge() {
-    FileRange base = new FileRangeImpl(2000, 1000);
+    FileRange base = FileRange.createFileRange(2000, 1000);
     CombinedFileRange mergeBase = new CombinedFileRange(2000, 3000, base);
 
     // test when the gap between is too big
     assertFalse("Large gap ranges shouldn't get merged", mergeBase.merge(5000, 6000,
-        new FileRangeImpl(5000, 1000), 2000, 4000));
+        FileRange.createFileRange(5000, 1000), 2000, 4000));
     assertEquals("Number of ranges in merged range shouldn't increase",
             1, mergeBase.getUnderlying().size());
     assertEquals("post merge offset", 2000, mergeBase.getOffset());
@@ -109,7 +107,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
 
     // test when the total size gets exceeded
     assertFalse("Large size ranges shouldn't get merged", mergeBase.merge(5000, 6000,
-        new FileRangeImpl(5000, 1000), 2001, 3999));
+        FileRange.createFileRange(5000, 1000), 2001, 3999));
     assertEquals("Number of ranges in merged range shouldn't increase",
             1, mergeBase.getUnderlying().size());
     assertEquals("post merge offset", 2000, mergeBase.getOffset());
@@ -117,7 +115,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
 
     // test when the merge works
     assertTrue("ranges should get merged ", mergeBase.merge(5000, 6000,
-        new FileRangeImpl(5000, 1000), 2001, 4000));
+        FileRange.createFileRange(5000, 1000), 2001, 4000));
     assertEquals("post merge size", 2, mergeBase.getUnderlying().size());
     assertEquals("post merge offset", 2000, mergeBase.getOffset());
     assertEquals("post merge length", 4000, mergeBase.getLength());
@@ -127,7 +125,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
     assertEquals(200, mergeBase.getOffset());
     assertEquals(100, mergeBase.getLength());
     assertTrue("ranges should get merged ", mergeBase.merge(500, 600,
-        new FileRangeImpl(5000, 1000), 201, 400));
+        FileRange.createFileRange(5000, 1000), 201, 400));
     assertEquals("post merge size", 2, mergeBase.getUnderlying().size());
     assertEquals("post merge offset", 200, mergeBase.getOffset());
     assertEquals("post merge length", 400, mergeBase.getLength());
@@ -136,42 +134,58 @@ public class TestVectoredReadUtils extends HadoopTestBase {
   @Test
   public void testSortAndMerge() {
     List<FileRange> input = Arrays.asList(
-        new FileRangeImpl(3000, 100),
-        new FileRangeImpl(2100, 100),
-        new FileRangeImpl(1000, 100)
+        FileRange.createFileRange(3000, 100),
+        FileRange.createFileRange(2100, 100),
+        FileRange.createFileRange(1000, 100)
         );
     assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
-    List<CombinedFileRange> outputList = VectoredReadUtils.sortAndMergeRanges(
-        input, 100, 1001, 2500);
-    assertEquals("merged range size", 1, outputList.size());
+    List<CombinedFileRange> outputList = VectoredReadUtils.mergeSortedRanges(
+            Arrays.asList(sortRanges(input)), 100, 1001, 2500);
+    Assertions.assertThat(outputList)
+            .describedAs("merged range size")
+            .hasSize(1);
     CombinedFileRange output = outputList.get(0);
-    assertEquals("merged range underlying size", 3, output.getUnderlying().size());
+    Assertions.assertThat(output.getUnderlying())
+            .describedAs("merged range underlying size")
+            .hasSize(3);
     assertEquals("range[1000,3100)", output.toString());
     assertTrue("merged output ranges are disjoint",
             VectoredReadUtils.isOrderedDisjoint(outputList, 100, 800));
 
     // the minSeek doesn't allow the first two to merge
-    assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 100, 1000));
-    outputList = VectoredReadUtils.sortAndMergeRanges(input, 100, 1000, 2100);
-    assertEquals("merged range size", 2, outputList.size());
+    assertFalse("Ranges are non disjoint",
+            VectoredReadUtils.isOrderedDisjoint(input, 100, 1000));
+    outputList = VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(input)),
+            100, 1000, 2100);
+    Assertions.assertThat(outputList)
+            .describedAs("merged range size")
+            .hasSize(2);
     assertEquals("range[1000,1100)", outputList.get(0).toString());
     assertEquals("range[2100,3100)", outputList.get(1).toString());
     assertTrue("merged output ranges are disjoint",
             VectoredReadUtils.isOrderedDisjoint(outputList, 100, 1000));
 
     // the maxSize doesn't allow the third range to merge
-    assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
-    outputList = VectoredReadUtils.sortAndMergeRanges(input, 100, 1001, 2099);
-    assertEquals("merged range size", 2, outputList.size());
+    assertFalse("Ranges are non disjoint",
+            VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
+    outputList = VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(input)),
+            100, 1001, 2099);
+    Assertions.assertThat(outputList)
+            .describedAs("merged range size")
+            .hasSize(2);
     assertEquals("range[1000,2200)", outputList.get(0).toString());
     assertEquals("range[3000,3100)", outputList.get(1).toString());
     assertTrue("merged output ranges are disjoint",
             VectoredReadUtils.isOrderedDisjoint(outputList, 100, 800));
 
     // test the round up and round down (the maxSize doesn't allow any merges)
-    assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 16, 700));
-    outputList = VectoredReadUtils.sortAndMergeRanges(input, 16, 1001, 100);
-    assertEquals("merged range size", 3, outputList.size());
+    assertFalse("Ranges are non disjoint",
+            VectoredReadUtils.isOrderedDisjoint(input, 16, 700));
+    outputList = VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(input)),
+            16, 1001, 100);
+    Assertions.assertThat(outputList)
+            .describedAs("merged range size")
+            .hasSize(3);
     assertEquals("range[992,1104)", outputList.get(0).toString());
     assertEquals("range[2096,2208)", outputList.get(1).toString());
     assertEquals("range[2992,3104)", outputList.get(2).toString());
@@ -182,26 +196,35 @@ public class TestVectoredReadUtils extends HadoopTestBase {
   @Test
   public void testSortAndMergeMoreCases() throws Exception {
     List<FileRange> input = Arrays.asList(
-            new FileRangeImpl(3000, 110),
-            new FileRangeImpl(3000, 100),
-            new FileRangeImpl(2100, 100),
-            new FileRangeImpl(1000, 100)
+            FileRange.createFileRange(3000, 110),
+            FileRange.createFileRange(3000, 100),
+            FileRange.createFileRange(2100, 100),
+            FileRange.createFileRange(1000, 100)
     );
-    assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
-    List<CombinedFileRange> outputList = VectoredReadUtils.sortAndMergeRanges(
-            input, 1, 1001, 2500);
-    assertEquals("merged range size", 1, outputList.size());
+    assertFalse("Ranges are non disjoint",
+            VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
+    List<CombinedFileRange> outputList = VectoredReadUtils.mergeSortedRanges(
+            Arrays.asList(sortRanges(input)), 1, 1001, 2500);
+    Assertions.assertThat(outputList)
+            .describedAs("merged range size")
+            .hasSize(1);
     CombinedFileRange output = outputList.get(0);
-    assertEquals("merged range underlying size", 4, output.getUnderlying().size());
+    Assertions.assertThat(output.getUnderlying())
+            .describedAs("merged range underlying size")
+            .hasSize(4);
     assertEquals("range[1000,3110)", output.toString());
     assertTrue("merged output ranges are disjoint",
             VectoredReadUtils.isOrderedDisjoint(outputList, 1, 800));
 
-    outputList = VectoredReadUtils.sortAndMergeRanges(
-            input, 100, 1001, 2500);
-    assertEquals("merged range size", 1, outputList.size());
+    outputList = VectoredReadUtils.mergeSortedRanges(
+            Arrays.asList(sortRanges(input)), 100, 1001, 2500);
+    Assertions.assertThat(outputList)
+            .describedAs("merged range size")
+            .hasSize(1);
     output = outputList.get(0);
-    assertEquals("merged range underlying size", 4, output.getUnderlying().size());
+    Assertions.assertThat(output.getUnderlying())
+            .describedAs("merged range underlying size")
+            .hasSize(4);
     assertEquals("range[1000,3200)", output.toString());
     assertTrue("merged output ranges are disjoint",
             VectoredReadUtils.isOrderedDisjoint(outputList, 1, 800));
@@ -211,9 +234,9 @@ public class TestVectoredReadUtils extends HadoopTestBase {
   @Test
   public void testMaxSizeZeroDisablesMering() throws Exception {
     List<FileRange> randomRanges = Arrays.asList(
-            new FileRangeImpl(3000, 110),
-            new FileRangeImpl(3000, 100),
-            new FileRangeImpl(2100, 100)
+            FileRange.createFileRange(3000, 110),
+            FileRange.createFileRange(3000, 100),
+            FileRange.createFileRange(2100, 100)
     );
     assertEqualRangeCountsAfterMerging(randomRanges, 1, 1, 0);
     assertEqualRangeCountsAfterMerging(randomRanges, 1, 0, 0);
@@ -225,7 +248,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
                                                   int minimumSeek,
                                                   int maxSize) {
     List<CombinedFileRange> combinedFileRanges = VectoredReadUtils
-            .sortAndMergeRanges(inputRanges, chunkSize, minimumSeek, maxSize);
+            .mergeSortedRanges(inputRanges, chunkSize, minimumSeek, maxSize);
     Assertions.assertThat(combinedFileRanges)
             .describedAs("Mismatch in number of ranges post merging")
             .hasSize(inputRanges.size());
@@ -251,7 +274,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
     }).when(stream).readFully(ArgumentMatchers.anyLong(),
                               ArgumentMatchers.any(ByteBuffer.class));
     CompletableFuture<ByteBuffer> result =
-        VectoredReadUtils.readRangeFrom(stream, new FileRangeImpl(1000, 100),
+        VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100),
         ByteBuffer::allocate);
     assertFutureCompletedSuccessfully(result);
     ByteBuffer buffer = result.get();
@@ -267,7 +290,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
         .when(stream).readFully(ArgumentMatchers.anyLong(),
                                 ArgumentMatchers.any(ByteBuffer.class));
     result =
-        VectoredReadUtils.readRangeFrom(stream, new FileRangeImpl(1000, 100),
+        VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100),
             ByteBuffer::allocate);
     assertFutureFailedExceptionally(result);
   }
@@ -286,7 +309,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
         ArgumentMatchers.any(), ArgumentMatchers.anyInt(),
         ArgumentMatchers.anyInt());
     CompletableFuture<ByteBuffer> result =
-        VectoredReadUtils.readRangeFrom(stream, new FileRangeImpl(1000, 100),
+        VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100),
             allocate);
     assertFutureCompletedSuccessfully(result);
     ByteBuffer buffer = result.get();
@@ -303,7 +326,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
         ArgumentMatchers.any(), ArgumentMatchers.anyInt(),
         ArgumentMatchers.anyInt());
     result =
-        VectoredReadUtils.readRangeFrom(stream, new FileRangeImpl(1000, 100),
+        VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100),
             ByteBuffer::allocate);
     assertFutureFailedExceptionally(result);
   }
@@ -328,9 +351,9 @@ public class TestVectoredReadUtils extends HadoopTestBase {
 
   @Test
   public void testReadVectored() throws Exception {
-    List<FileRange> input = Arrays.asList(new FileRangeImpl(0, 100),
-        new FileRangeImpl(100_000, 100),
-        new FileRangeImpl(200_000, 100));
+    List<FileRange> input = Arrays.asList(FileRange.createFileRange(0, 100),
+        FileRange.createFileRange(100_000, 100),
+        FileRange.createFileRange(200_000, 100));
     Stream stream = Mockito.mock(Stream.class);
     Mockito.doAnswer(invocation -> {
       fillBuffer(invocation.getArgument(1));
@@ -338,31 +361,11 @@ public class TestVectoredReadUtils extends HadoopTestBase {
     }).when(stream).readFully(ArgumentMatchers.anyLong(),
         ArgumentMatchers.any(ByteBuffer.class));
     // should not merge the ranges
-    VectoredReadUtils.readVectored(stream, input, ByteBuffer::allocate, 100, 100);
+    VectoredReadUtils.readVectored(stream, input, ByteBuffer::allocate);
     Mockito.verify(stream, Mockito.times(3))
         .readFully(ArgumentMatchers.anyLong(), ArgumentMatchers.any(ByteBuffer.class));
     for(int b=0; b < input.size(); ++b) {
       validateBuffer("buffer " + b, input.get(b).getData().get(), 0);
     }
   }
-
-  @Test
-  public void testReadVectoredMerge() throws Exception {
-    List<FileRange> input = Arrays.asList(new FileRangeImpl(2000, 100),
-        new FileRangeImpl(1000, 100),
-        new FileRangeImpl(0, 100));
-    Stream stream = Mockito.mock(Stream.class);
-    Mockito.doAnswer(invocation -> {
-      fillBuffer(invocation.getArgument(1));
-      return null;
-    }).when(stream).readFully(ArgumentMatchers.anyLong(),
-        ArgumentMatchers.any(ByteBuffer.class));
-    // should merge the ranges into a single read
-    VectoredReadUtils.readVectored(stream, input, ByteBuffer::allocate, 1000, 2100);
-    Mockito.verify(stream, Mockito.times(1))
-        .readFully(ArgumentMatchers.anyLong(), ArgumentMatchers.any(ByteBuffer.class));
-    for(int b=0; b < input.size(); ++b) {
-      validateBuffer("buffer " + b, input.get(b).getData().get(), (2 - b) * 1000);
-    }
-  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
index e8c86b5dbbc..77bcc496ff4 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.fs.contract;
 
 import java.io.EOFException;
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -38,13 +37,18 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileRange;
-import org.apache.hadoop.fs.FileRangeImpl;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.impl.FutureIOSupport;
+import org.apache.hadoop.io.WeakReferencedElasticByteBufferPool;
+import org.apache.hadoop.test.LambdaTestUtils;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertCapabilities;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDatasetEquals;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.returnBuffersToPoolPostRead;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
 
 @RunWith(Parameterized.class)
@@ -53,11 +57,14 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
   private static final Logger LOG = LoggerFactory.getLogger(AbstractContractVectoredReadTest.class);
 
   public static final int DATASET_LEN = 64 * 1024;
-  private static final byte[] DATASET = ContractTestUtils.dataset(DATASET_LEN, 'a', 32);
+  protected static final byte[] DATASET = ContractTestUtils.dataset(DATASET_LEN, 'a', 32);
   protected static final String VECTORED_READ_FILE_NAME = "vectored_file.txt";
 
   private final IntFunction<ByteBuffer> allocate;
 
+  private final WeakReferencedElasticByteBufferPool pool =
+          new WeakReferencedElasticByteBufferPool();
+
   private final String bufferType;
 
   @Parameterized.Parameters(name = "Buffer type : {0}")
@@ -67,8 +74,14 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
 
   public AbstractContractVectoredReadTest(String bufferType) {
     this.bufferType = bufferType;
-    this.allocate = "array".equals(bufferType) ?
-            ByteBuffer::allocate : ByteBuffer::allocateDirect;
+    this.allocate = value -> {
+      boolean isDirect = !"array".equals(bufferType);
+      return pool.getBuffer(isDirect, value);
+    };
+  }
+
+  public IntFunction<ByteBuffer> getAllocate() {
+    return allocate;
   }
 
   @Override
@@ -79,12 +92,27 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
     createFile(fs, path, true, DATASET);
   }
 
+  @Override
+  public void teardown() throws Exception {
+    super.teardown();
+    pool.release();
+  }
+
+  @Test
+  public void testVectoredReadCapability() throws Exception {
+    FileSystem fs = getFileSystem();
+    String[] vectoredReadCapability = new String[]{StreamCapabilities.VECTOREDIO};
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      assertCapabilities(in, vectoredReadCapability, null);
+    }
+  }
+
   @Test
   public void testVectoredReadMultipleRanges() throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
-      FileRange fileRange = new FileRangeImpl(i * 100, 100);
+      FileRange fileRange = FileRange.createFileRange(i * 100, 100);
       fileRanges.add(fileRange);
     }
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
@@ -98,6 +126,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
       combinedFuture.get();
 
       validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
     }
   }
 
@@ -105,7 +134,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
   public void testVectoredReadAndReadFully()  throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(100, 100));
+    fileRanges.add(FileRange.createFileRange(100, 100));
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
       byte[] readFullRes = new byte[100];
@@ -114,6 +143,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
       Assertions.assertThat(vecRes)
               .describedAs("Result from vectored read and readFully must match")
               .isEqualByComparingTo(ByteBuffer.wrap(readFullRes));
+      returnBuffersToPoolPostRead(fileRanges, pool);
     }
   }
 
@@ -125,12 +155,13 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
   public void testDisjointRanges() throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(0, 100));
-    fileRanges.add(new FileRangeImpl(4 * 1024 + 101, 100));
-    fileRanges.add(new FileRangeImpl(16 * 1024 + 101, 100));
+    fileRanges.add(FileRange.createFileRange(0, 100));
+    fileRanges.add(FileRange.createFileRange(4_000 + 101, 100));
+    fileRanges.add(FileRange.createFileRange(16_000 + 101, 100));
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
       validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
     }
   }
 
@@ -142,12 +173,13 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
   public void testAllRangesMergedIntoOne() throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(0, 100));
-    fileRanges.add(new FileRangeImpl(4 *1024 - 101, 100));
-    fileRanges.add(new FileRangeImpl(8*1024 - 101, 100));
+    fileRanges.add(FileRange.createFileRange(0, 100));
+    fileRanges.add(FileRange.createFileRange(4_000 - 101, 100));
+    fileRanges.add(FileRange.createFileRange(8_000 - 101, 100));
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
       validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
     }
   }
 
@@ -159,44 +191,80 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
   public void testSomeRangesMergedSomeUnmerged() throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(8*1024, 100));
-    fileRanges.add(new FileRangeImpl(14*1024, 100));
-    fileRanges.add(new FileRangeImpl(10*1024, 100));
-    fileRanges.add(new FileRangeImpl(2 *1024 - 101, 100));
-    fileRanges.add(new FileRangeImpl(40*1024, 1024));
-    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+    fileRanges.add(FileRange.createFileRange(8 * 1024, 100));
+    fileRanges.add(FileRange.createFileRange(14 * 1024, 100));
+    fileRanges.add(FileRange.createFileRange(10 * 1024, 100));
+    fileRanges.add(FileRange.createFileRange(2 * 1024 - 101, 100));
+    fileRanges.add(FileRange.createFileRange(40 * 1024, 1024));
+    FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME));
+    CompletableFuture<FSDataInputStream> builder =
+            fs.openFile(path(VECTORED_READ_FILE_NAME))
+                    .withFileStatus(fileStatus)
+                    .build();
+    try (FSDataInputStream in = builder.get()) {
+      in.readVectored(fileRanges, allocate);
+      validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
+    }
+  }
+
+  @Test
+  public void testOverlappingRanges() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = getSampleOverlappingRanges();
+    FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME));
+    CompletableFuture<FSDataInputStream> builder =
+            fs.openFile(path(VECTORED_READ_FILE_NAME))
+                    .withFileStatus(fileStatus)
+                    .build();
+    try (FSDataInputStream in = builder.get()) {
       in.readVectored(fileRanges, allocate);
       validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
     }
   }
 
   @Test
   public void testSameRanges() throws Exception {
+    // Same ranges are special case of overlapping only.
     FileSystem fs = getFileSystem();
-    List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(8*1024, 1000));
-    fileRanges.add(new FileRangeImpl(8*1024, 1000));
-    fileRanges.add(new FileRangeImpl(8*1024, 1000));
+    List<FileRange> fileRanges = getSampleSameRanges();
     CompletableFuture<FSDataInputStream> builder =
             fs.openFile(path(VECTORED_READ_FILE_NAME))
                     .build();
     try (FSDataInputStream in = builder.get()) {
       in.readVectored(fileRanges, allocate);
       validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
     }
   }
 
   @Test
-  public void testOverlappingRanges() throws Exception {
+  public void testSomeRandomNonOverlappingRanges() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(FileRange.createFileRange(500, 100));
+    fileRanges.add(FileRange.createFileRange(1000, 200));
+    fileRanges.add(FileRange.createFileRange(50, 10));
+    fileRanges.add(FileRange.createFileRange(10, 5));
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
+    }
+  }
+
+  @Test
+  public void testConsecutiveRanges() throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(0, 1000));
-    fileRanges.add(new FileRangeImpl(90, 900));
-    fileRanges.add(new FileRangeImpl(50, 900));
-    fileRanges.add(new FileRangeImpl(10, 980));
+    fileRanges.add(FileRange.createFileRange(500, 100));
+    fileRanges.add(FileRange.createFileRange(600, 200));
+    fileRanges.add(FileRange.createFileRange(800, 100));
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
       validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
     }
   }
 
@@ -204,7 +272,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
   public void testEOFRanges()  throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(DATASET_LEN, 100));
+    fileRanges.add(FileRange.createFileRange(DATASET_LEN, 100));
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
       for (FileRange res : fileRanges) {
@@ -227,22 +295,22 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
   public void testNegativeLengthRange()  throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(0, -50));
-    testExceptionalVectoredRead(fs, fileRanges, "Exception is expected");
+    fileRanges.add(FileRange.createFileRange(0, -50));
+    verifyExceptionalVectoredRead(fs, fileRanges, IllegalArgumentException.class);
   }
 
   @Test
   public void testNegativeOffsetRange()  throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(-1, 50));
-    testExceptionalVectoredRead(fs, fileRanges, "Exception is expected");
+    fileRanges.add(FileRange.createFileRange(-1, 50));
+    verifyExceptionalVectoredRead(fs, fileRanges, EOFException.class);
   }
 
   @Test
   public void testNormalReadAfterVectoredRead() throws Exception {
     FileSystem fs = getFileSystem();
-    List<FileRange> fileRanges = createSomeOverlappingRanges();
+    List<FileRange> fileRanges = createSampleNonOverlappingRanges();
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
       // read starting 200 bytes
@@ -254,13 +322,14 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
               .describedAs("Vectored read shouldn't change file pointer.")
               .isEqualTo(200);
       validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
     }
   }
 
   @Test
   public void testVectoredReadAfterNormalRead() throws Exception {
     FileSystem fs = getFileSystem();
-    List<FileRange> fileRanges = createSomeOverlappingRanges();
+    List<FileRange> fileRanges = createSampleNonOverlappingRanges();
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       // read starting 200 bytes
       byte[] res = new byte[200];
@@ -272,43 +341,66 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
               .isEqualTo(200);
       in.readVectored(fileRanges, allocate);
       validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
     }
   }
 
   @Test
   public void testMultipleVectoredReads() throws Exception {
     FileSystem fs = getFileSystem();
-    List<FileRange> fileRanges1 = createSomeOverlappingRanges();
-    List<FileRange> fileRanges2 = createSomeOverlappingRanges();
+    List<FileRange> fileRanges1 = createSampleNonOverlappingRanges();
+    List<FileRange> fileRanges2 = createSampleNonOverlappingRanges();
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges1, allocate);
       in.readVectored(fileRanges2, allocate);
       validateVectoredReadResult(fileRanges2, DATASET);
       validateVectoredReadResult(fileRanges1, DATASET);
+      returnBuffersToPoolPostRead(fileRanges1, pool);
+      returnBuffersToPoolPostRead(fileRanges2, pool);
     }
   }
 
-  protected List<FileRange> createSomeOverlappingRanges() {
+  protected List<FileRange> createSampleNonOverlappingRanges() {
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(FileRange.createFileRange(0, 100));
+    fileRanges.add(FileRange.createFileRange(110, 50));
+    return fileRanges;
+  }
+
+  protected List<FileRange> getSampleSameRanges() {
     List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(0, 100));
-    fileRanges.add(new FileRangeImpl(90, 50));
+    fileRanges.add(FileRange.createFileRange(8_000, 1000));
+    fileRanges.add(FileRange.createFileRange(8_000, 1000));
+    fileRanges.add(FileRange.createFileRange(8_000, 1000));
     return fileRanges;
   }
 
+  protected List<FileRange> getSampleOverlappingRanges() {
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(FileRange.createFileRange(100, 500));
+    fileRanges.add(FileRange.createFileRange(400, 500));
+    return fileRanges;
+  }
 
-  protected void testExceptionalVectoredRead(FileSystem fs,
-                                             List<FileRange> fileRanges,
-                                             String s) throws IOException {
-    boolean exRaised = false;
-    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
-      // Can we intercept here as done in S3 tests ??
-      in.readVectored(fileRanges, allocate);
-    } catch (EOFException | IllegalArgumentException ex) {
-      // expected.
-      exRaised = true;
+  /**
+   * Validate that exceptions must be thrown during a vectored
+   * read operation with specific input ranges.
+   * @param fs FileSystem instance.
+   * @param fileRanges input file ranges.
+   * @param clazz type of exception expected.
+   * @throws Exception any other IOE.
+   */
+  protected <T extends Throwable> void verifyExceptionalVectoredRead(
+          FileSystem fs,
+          List<FileRange> fileRanges,
+          Class<T> clazz) throws Exception {
+
+    CompletableFuture<FSDataInputStream> builder =
+            fs.openFile(path(VECTORED_READ_FILE_NAME))
+                    .build();
+    try (FSDataInputStream in = builder.get()) {
+      LambdaTestUtils.intercept(clazz,
+          () -> in.readVectored(fileRanges, allocate));
     }
-    Assertions.assertThat(exRaised)
-            .describedAs(s)
-            .isTrue();
   }
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
index 8c071282dc5..b61abddd434 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathCapabilities;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.io.ByteBufferPool;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.functional.RemoteIterators;
 import org.apache.hadoop.util.functional.FutureIO;
@@ -1137,6 +1138,25 @@ public class ContractTestUtils extends Assert {
     }
   }
 
+  /**
+   * Utility to return buffers back to the pool once all
+   * data has been read for each file range.
+   * @param fileRanges list of file range.
+   * @param pool buffer pool.
+   * @throws IOException any IOE
+   * @throws TimeoutException ideally this should never occur.
+   */
+  public static void returnBuffersToPoolPostRead(List<FileRange> fileRanges,
+                                                 ByteBufferPool pool)
+          throws IOException, TimeoutException {
+    for (FileRange range : fileRanges) {
+      ByteBuffer buffer = FutureIO.awaitFuture(range.getData(),
+              VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS,
+              TimeUnit.SECONDS);
+      pool.putBuffer(buffer);
+    }
+  }
+
 
   /**
    * Assert that the data read matches the dataset at the given offset.
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java
index 099e3b946d1..5d6ca3f8f0c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java
@@ -18,9 +18,26 @@
 
 package org.apache.hadoop.fs.contract.localfs;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 public class TestLocalFSContractVectoredRead extends AbstractContractVectoredReadTest {
 
@@ -32,4 +49,38 @@ public class TestLocalFSContractVectoredRead extends AbstractContractVectoredRea
   protected AbstractFSContract createContract(Configuration conf) {
     return new LocalFSContract(conf);
   }
+
+  @Test
+  public void testChecksumValidationDuringVectoredRead() throws Exception {
+    Path testPath = path("big_range_checksum");
+    LocalFileSystem localFs = (LocalFileSystem) getFileSystem();
+    final byte[] datasetCorrect = ContractTestUtils.dataset(DATASET_LEN, 'a', 32);
+    try (FSDataOutputStream out = localFs.create(testPath, true)){
+      out.write(datasetCorrect);
+    }
+    Path checksumPath = localFs.getChecksumFile(testPath);
+    Assertions.assertThat(localFs.exists(checksumPath))
+            .describedAs("Checksum file should be present")
+            .isTrue();
+    CompletableFuture<FSDataInputStream> fis = localFs.openFile(testPath).build();
+    List<FileRange> someRandomRanges = new ArrayList<>();
+    someRandomRanges.add(FileRange.createFileRange(10, 1024));
+    someRandomRanges.add(FileRange.createFileRange(1025, 1024));
+    try (FSDataInputStream in = fis.get()){
+      in.readVectored(someRandomRanges, getAllocate());
+      validateVectoredReadResult(someRandomRanges, datasetCorrect);
+    }
+    final byte[] datasetCorrupted = ContractTestUtils.dataset(DATASET_LEN, 'a', 64);
+    try (FSDataOutputStream out = localFs.getRaw().create(testPath, true)){
+      out.write(datasetCorrupted);
+    }
+    CompletableFuture<FSDataInputStream> fisN = localFs.openFile(testPath).build();
+    try (FSDataInputStream in = fisN.get()){
+      in.readVectored(someRandomRanges, getAllocate());
+      // Expect checksum exception when data is updated directly through
+      // raw local fs instance.
+      intercept(ChecksumException.class,
+          () -> validateVectoredReadResult(someRandomRanges, datasetCorrupted));
+    }
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index 9d87f26c3c0..3069f172891 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -22,11 +22,13 @@ import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.io.EOFException;
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.net.SocketTimeoutException;
 import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.IntFunction;
 
 import com.amazonaws.services.s3.model.GetObjectRequest;
@@ -46,8 +48,7 @@ import org.apache.hadoop.fs.FileRange;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.impl.CombinedFileRange;
-import org.apache.hadoop.fs.impl.FutureIOSupport;
-import org.apache.hadoop.fs.impl.VectoredReadUtils;
+import org.apache.hadoop.fs.VectoredReadUtils;
 import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
 import org.apache.hadoop.fs.statistics.DurationTracker;
@@ -59,9 +60,9 @@ import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
 import static java.util.Objects.requireNonNull;
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
-import static org.apache.hadoop.fs.impl.VectoredReadUtils.isOrderedDisjoint;
-import static org.apache.hadoop.fs.impl.VectoredReadUtils.sliceTo;
-import static org.apache.hadoop.fs.impl.VectoredReadUtils.sortAndMergeRanges;
+import static org.apache.hadoop.fs.VectoredReadUtils.isOrderedDisjoint;
+import static org.apache.hadoop.fs.VectoredReadUtils.mergeSortedRanges;
+import static org.apache.hadoop.fs.VectoredReadUtils.validateNonOverlappingAndReturnSortedRanges;
 import static org.apache.hadoop.fs.s3a.Invoker.onceTrackingDuration;
 import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
 import static org.apache.hadoop.util.StringUtils.toLowerCase;
@@ -107,6 +108,13 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    */
   private static final int TMP_BUFFER_MAX_SIZE = 64 * 1024;
 
+  /**
+   * Atomic boolean variable to stop all ongoing vectored read operation
+   * for this input stream. This will be set to true when the stream is
+   * closed or unbuffer is called.
+   */
+  private final AtomicBoolean stopVectoredIOOperations = new AtomicBoolean(false);
+
   /**
    * This is the public position; the one set in {@link #seek(long)}
    * and returned in {@link #getPos()}.
@@ -589,6 +597,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     if (!closed) {
       closed = true;
       try {
+        stopVectoredIOOperations.set(true);
         // close or abort the stream; blocking
         awaitFuture(closeStream("close() operation", false, true));
         LOG.debug("Statistics of stream {}\n{}", key, streamStatistics);
@@ -940,31 +949,32 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
 
     LOG.debug("Starting vectored read on path {} for ranges {} ", pathStr, ranges);
     checkNotClosed();
+    if (stopVectoredIOOperations.getAndSet(false)) {
+      LOG.debug("Reinstating vectored read operation for path {} ", pathStr);
+    }
+    List<? extends FileRange> sortedRanges = validateNonOverlappingAndReturnSortedRanges(ranges);
     for (FileRange range : ranges) {
       validateRangeRequest(range);
       CompletableFuture<ByteBuffer> result = new CompletableFuture<>();
       range.setData(result);
     }
 
-    if (isOrderedDisjoint(ranges, 1, minSeekForVectorReads())) {
+    if (isOrderedDisjoint(sortedRanges, 1, minSeekForVectorReads())) {
       LOG.debug("Not merging the ranges as they are disjoint");
-      for(FileRange range: ranges) {
+      for (FileRange range: sortedRanges) {
         ByteBuffer buffer = allocate.apply(range.getLength());
         unboundedThreadPool.submit(() -> readSingleRange(range, buffer));
       }
     } else {
       LOG.debug("Trying to merge the ranges as they are not disjoint");
-      List<CombinedFileRange> combinedFileRanges = sortAndMergeRanges(ranges,
+      List<CombinedFileRange> combinedFileRanges = mergeSortedRanges(sortedRanges,
               1, minSeekForVectorReads(),
               maxReadSizeForVectorReads());
       LOG.debug("Number of original ranges size {} , Number of combined ranges {} ",
               ranges.size(), combinedFileRanges.size());
-      for(CombinedFileRange combinedFileRange: combinedFileRanges) {
-        CompletableFuture<ByteBuffer> result = new CompletableFuture<>();
-        ByteBuffer buffer = allocate.apply(combinedFileRange.getLength());
-        combinedFileRange.setData(result);
+      for (CombinedFileRange combinedFileRange: combinedFileRanges) {
         unboundedThreadPool.submit(
-            () -> readCombinedRangeAndUpdateChildren(combinedFileRange, buffer));
+            () -> readCombinedRangeAndUpdateChildren(combinedFileRange, allocate));
       }
     }
     LOG.debug("Finished submitting vectored read to threadpool" +
@@ -972,58 +982,102 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   }
 
   /**
-   * Read data in the combinedFileRange and update data in buffers
-   * of all underlying ranges.
-   * @param combinedFileRange combined range.
-   * @param buffer combined buffer.
+   * Read the data from S3 for the bigger combined file range and update all the
+   * underlying ranges.
+   * @param combinedFileRange big combined file range.
+   * @param allocate method to create byte buffers to hold result data.
    */
   private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRange,
-                                                  ByteBuffer buffer) {
-    // Not putting read single range call inside try block as
-    // exception if any occurred during this call will be raised
-    // during awaitFuture call while getting the combined buffer.
-    readSingleRange(combinedFileRange, buffer);
+                                                  IntFunction<ByteBuffer> allocate) {
+    LOG.debug("Start reading combined range {} from path {} ", combinedFileRange, pathStr);
+    // This reference is must be kept till all buffers are populated as this is a
+    // finalizable object which closes the internal stream when gc triggers.
+    S3Object objectRange = null;
+    S3ObjectInputStream objectContent = null;
     try {
-      // In case of single range we return the original byte buffer else
-      // we return slice byte buffers for each child ranges.
-      ByteBuffer combinedBuffer = FutureIOSupport.awaitFuture(combinedFileRange.getData());
-      if (combinedFileRange.getUnderlying().size() == 1) {
-        combinedFileRange.getUnderlying().get(0).getData().complete(combinedBuffer);
-      } else {
-        for (FileRange child : combinedFileRange.getUnderlying()) {
-          updateOriginalRange(child, combinedBuffer, combinedFileRange);
-        }
+      checkIfVectoredIOStopped();
+      final String operationName = "readCombinedFileRange";
+      objectRange = getS3Object(operationName,
+              combinedFileRange.getOffset(),
+              combinedFileRange.getLength());
+      objectContent = objectRange.getObjectContent();
+      if (objectContent == null) {
+        throw new PathIOException(uri,
+                "Null IO stream received during " + operationName);
       }
+      populateChildBuffers(combinedFileRange, objectContent, allocate);
     } catch (Exception ex) {
-      LOG.warn("Exception occurred while reading combined range from file {}", pathStr, ex);
+      LOG.debug("Exception while reading a range {} from path {} ", combinedFileRange, pathStr, ex);
       for(FileRange child : combinedFileRange.getUnderlying()) {
         child.getData().completeExceptionally(ex);
       }
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, objectRange, objectContent);
+    }
+    LOG.debug("Finished reading range {} from path {} ", combinedFileRange, pathStr);
+  }
+
+  /**
+   * Populate underlying buffers of the child ranges.
+   * @param combinedFileRange big combined file range.
+   * @param objectContent data from s3.
+   * @param allocate method to allocate child byte buffers.
+   * @throws IOException any IOE.
+   */
+  private void populateChildBuffers(CombinedFileRange combinedFileRange,
+                                    S3ObjectInputStream objectContent,
+                                    IntFunction<ByteBuffer> allocate) throws IOException {
+    // If the combined file range just contains a single child
+    // range, we only have to fill that one child buffer else
+    // we drain the intermediate data between consecutive ranges
+    // and fill the buffers one by one.
+    if (combinedFileRange.getUnderlying().size() == 1) {
+      FileRange child = combinedFileRange.getUnderlying().get(0);
+      ByteBuffer buffer = allocate.apply(child.getLength());
+      populateBuffer(child.getLength(), buffer, objectContent);
+      child.getData().complete(buffer);
+    } else {
+      FileRange prev = null;
+      for (FileRange child : combinedFileRange.getUnderlying()) {
+        if (prev != null && prev.getOffset() + prev.getLength() < child.getOffset()) {
+          long drainQuantity = child.getOffset() - prev.getOffset() - prev.getLength();
+          drainUnnecessaryData(objectContent, drainQuantity);
+        }
+        ByteBuffer buffer = allocate.apply(child.getLength());
+        populateBuffer(child.getLength(), buffer, objectContent);
+        child.getData().complete(buffer);
+        prev = child;
+      }
     }
   }
 
   /**
-   * Update data in child range from combined range.
-   * @param child child range.
-   * @param combinedBuffer combined buffer.
-   * @param combinedFileRange combined range.
+   * Drain unnecessary data in between ranges.
+   * @param objectContent s3 data stream.
+   * @param drainQuantity how many bytes to drain.
+   * @throws IOException any IOE.
    */
-  private void updateOriginalRange(FileRange child,
-                                   ByteBuffer combinedBuffer,
-                                   CombinedFileRange combinedFileRange) {
-    LOG.trace("Start Filling original range [{}, {}) from combined range [{}, {}) ",
-            child.getOffset(), child.getLength(),
-            combinedFileRange.getOffset(), combinedFileRange.getLength());
-    ByteBuffer childBuffer = sliceTo(combinedBuffer, combinedFileRange.getOffset(), child);
-    child.getData().complete(childBuffer);
-    LOG.trace("End Filling original range [{}, {}) from combined range [{}, {}) ",
-            child.getOffset(), child.getLength(),
-            combinedFileRange.getOffset(), combinedFileRange.getLength());
+  private void drainUnnecessaryData(S3ObjectInputStream objectContent, long drainQuantity)
+          throws IOException {
+    int drainBytes = 0;
+    int readCount;
+    while (drainBytes < drainQuantity) {
+      if (drainBytes + DRAIN_BUFFER_SIZE <= drainQuantity) {
+        byte[] drainBuffer = new byte[DRAIN_BUFFER_SIZE];
+        readCount = objectContent.read(drainBuffer);
+      } else {
+        byte[] drainBuffer = new byte[(int) (drainQuantity - drainBytes)];
+        readCount = objectContent.read(drainBuffer);
+      }
+      drainBytes += readCount;
+    }
+    LOG.debug("{} bytes drained from stream ", drainBytes);
   }
 
   /**
-   *  // Check if we can use contentLength returned by http GET request.
    * Validates range parameters.
+   * In case of S3 we already have contentLength from the first GET request
+   * during an open file operation so failing fast here.
    * @param range requested range.
    * @throws EOFException end of file exception.
    */
@@ -1038,13 +1092,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   }
 
   /**
-   * TODO: Add retry in client.getObject(). not present in older reads why here??
-   * Okay retry is being done in the top layer during read.
-   * But if we do here in the top layer, one issue I am thinking is
-   * what if there is some error which happened during filling the buffer
-   * If we retry that old offsets of heap buffers can be overwritten ?
-   * I think retry should be only added in {@link S3AInputStream#getS3Object}
-   * Read data from S3 for this range and populate the bufffer.
+   * Read data from S3 for this range and populate the buffer.
    * @param range range of data to read.
    * @param buffer buffer to fill.
    */
@@ -1053,6 +1101,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     S3Object objectRange = null;
     S3ObjectInputStream objectContent = null;
     try {
+      checkIfVectoredIOStopped();
       long position = range.getOffset();
       int length = range.getLength();
       final String operationName = "readRange";
@@ -1089,6 +1138,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       int offset = 0;
       byte[] tmp = new byte[TMP_BUFFER_MAX_SIZE];
       while (readBytes < length) {
+        checkIfVectoredIOStopped();
         int currentLength = readBytes + TMP_BUFFER_MAX_SIZE < length ?
                 TMP_BUFFER_MAX_SIZE
                 : length - readBytes;
@@ -1103,7 +1153,15 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     }
   }
 
-  public void readByteArray(S3ObjectInputStream objectContent,
+  /**
+   * Read data into destination buffer from s3 object content.
+   * @param objectContent result from S3.
+   * @param dest destination buffer.
+   * @param offset start offset of dest buffer.
+   * @param length number of bytes to fill in dest.
+   * @throws IOException any IOE.
+   */
+  private void readByteArray(S3ObjectInputStream objectContent,
                             byte[] dest,
                             int offset,
                             int length) throws IOException {
@@ -1120,14 +1178,14 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   }
 
   /**
-   * Read data from S3 using a http request.
-   * This also handles if file has been changed while http call
-   * is getting executed. If file has been changed RemoteFileChangedException
-   * is thrown.
+   * Read data from S3 using a http request with retries.
+   * This also handles if file has been changed while the
+   * http call is getting executed. If the file has been
+   * changed RemoteFileChangedException is thrown.
    * @param operationName name of the operation for which get object on S3 is called.
    * @param position position of the object to be read from S3.
    * @param length length from position of the object to be read from S3.
-   * @return S3Object
+   * @return S3Object result s3 object.
    * @throws IOException exception if any.
    */
   private S3Object getS3Object(String operationName, long position,
@@ -1140,7 +1198,11 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     Invoker invoker = context.getReadInvoker();
     try {
       objectRange = invoker.retry(operationName, pathStr, true,
-          () -> client.getObject(request));
+        () -> {
+          checkIfVectoredIOStopped();
+          return client.getObject(request);
+        });
+
     } catch (IOException ex) {
       tracker.failed();
       throw ex;
@@ -1152,6 +1214,19 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     return objectRange;
   }
 
+  /**
+   * Check if vectored io operation has been stooped. This happens
+   * when the stream is closed or unbuffer is called.
+   * @throws InterruptedIOException throw InterruptedIOException such
+   *                                that all running vectored io is
+   *                                terminated thus releasing resources.
+   */
+  private void checkIfVectoredIOStopped() throws InterruptedIOException {
+    if (stopVectoredIOOperations.get()) {
+      throw new InterruptedIOException("Stream closed or unbuffer is called");
+    }
+  }
+
   /**
    * Access the input stream statistics.
    * This is for internal testing and may be removed without warning.
@@ -1237,10 +1312,15 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   /**
    * Closes the underlying S3 stream, and merges the {@link #streamStatistics}
    * instance associated with the stream.
+   * Also sets the {@code stopVectoredIOOperations} flag to true such that
+   * active vectored read operations are terminated. However termination of
+   * old vectored reads are not guaranteed if a new vectored read operation
+   * is initiated after unbuffer is called.
    */
   @Override
   public synchronized void unbuffer() {
     try {
+      stopVectoredIOOperations.set(true);
       closeStream("unbuffer()", false, false);
     } finally {
       streamStatistics.unbuffered();
@@ -1253,6 +1333,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     case StreamCapabilities.IOSTATISTICS:
     case StreamCapabilities.READAHEAD:
     case StreamCapabilities.UNBUFFER:
+    case StreamCapabilities.VECTOREDIO:
       return true;
     default:
       return false;
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java
index 0752e75d247..18a727dcdce 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.fs.contract.s3a;
 
+import java.io.EOFException;
+import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -26,14 +28,15 @@ import org.junit.Test;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileRange;
-import org.apache.hadoop.fs.FileRangeImpl;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
 import static org.apache.hadoop.test.MoreAsserts.assertEqual;
 
 public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTest {
@@ -55,8 +58,8 @@ public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTe
   public void testEOFRanges() throws Exception {
     FileSystem fs = getFileSystem();
     List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(DATASET_LEN, 100));
-    testExceptionalVectoredRead(fs, fileRanges, "EOFException is expected");
+    fileRanges.add(FileRange.createFileRange(DATASET_LEN, 100));
+    verifyExceptionalVectoredRead(fs, fileRanges, EOFException.class);
   }
 
   @Test
@@ -99,4 +102,58 @@ public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTe
       }
     }
   }
+
+  @Test
+  public void testStopVectoredIoOperationsCloseStream() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = createSampleNonOverlappingRanges();
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))){
+      in.readVectored(fileRanges, getAllocate());
+      in.close();
+      LambdaTestUtils.intercept(InterruptedIOException.class,
+          () -> validateVectoredReadResult(fileRanges, DATASET));
+    }
+    // reopening the stream should succeed.
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))){
+      in.readVectored(fileRanges, getAllocate());
+      validateVectoredReadResult(fileRanges, DATASET);
+    }
+  }
+
+  @Test
+  public void testStopVectoredIoOperationsUnbuffer() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = createSampleNonOverlappingRanges();
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))){
+      in.readVectored(fileRanges, getAllocate());
+      in.unbuffer();
+      LambdaTestUtils.intercept(InterruptedIOException.class,
+          () -> validateVectoredReadResult(fileRanges, DATASET));
+      // re-initiating the vectored reads after unbuffer should succeed.
+      in.readVectored(fileRanges, getAllocate());
+      validateVectoredReadResult(fileRanges, DATASET);
+    }
+
+  }
+
+  /**
+   * S3 vectored IO doesn't support overlapping ranges.
+   */
+  @Override
+  public void testOverlappingRanges() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = getSampleOverlappingRanges();
+    verifyExceptionalVectoredRead(fs, fileRanges, UnsupportedOperationException.class);
+  }
+
+  /**
+   * S3 vectored IO doesn't support overlapping ranges.
+   */
+  @Override
+  public void testSameRanges() throws Exception {
+    // Same ranges are special case of overlapping only.
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = getSampleSameRanges();
+    verifyExceptionalVectoredRead(fs, fileRanges, UnsupportedOperationException.class);
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
index 956e23a3f11..f8d47011de3 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileRange;
-import org.apache.hadoop.fs.FileRangeImpl;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -459,13 +458,13 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
   public void test_045_vectoredIOHugeFile() throws Throwable {
     assumeHugeFileExists();
     List<FileRange> rangeList = new ArrayList<>();
-    rangeList.add(new FileRangeImpl(5856368, 1167716));
-    rangeList.add(new FileRangeImpl(3520861, 1167700));
-    rangeList.add(new FileRangeImpl(8191913, 1167775));
-    rangeList.add(new FileRangeImpl(1520861, 1167700));
-    rangeList.add(new FileRangeImpl(2520861, 116770));
-    rangeList.add(new FileRangeImpl(9191913, 116770));
-    rangeList.add(new FileRangeImpl(2820861, 156770));
+    rangeList.add(FileRange.createFileRange(5856368, 116770));
+    rangeList.add(FileRange.createFileRange(3520861, 116770));
+    rangeList.add(FileRange.createFileRange(8191913, 116770));
+    rangeList.add(FileRange.createFileRange(1520861, 116770));
+    rangeList.add(FileRange.createFileRange(2520861, 116770));
+    rangeList.add(FileRange.createFileRange(9191913, 116770));
+    rangeList.add(FileRange.createFileRange(2820861, 156770));
     IntFunction<ByteBuffer> allocate = ByteBuffer::allocate;
     FileSystem fs = getFileSystem();
     CompletableFuture<FSDataInputStream> builder =
diff --git a/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java b/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java
index aaee951d72c..631842f78e2 100644
--- a/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java
+++ b/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileRange;
-import org.apache.hadoop.fs.FileRangeImpl;
+import org.apache.hadoop.fs.impl.FileRangeImpl;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
@@ -107,7 +107,7 @@ public class VectoredReadBenchmark {
     FSDataInputStream stream = fsChoice.fs.open(DATA_PATH);
     List<FileRange> ranges = new ArrayList<>();
     for(int m=0; m < 100; ++m) {
-      FileRangeImpl range = new FileRangeImpl(m * SEEK_SIZE, READ_SIZE);
+      FileRange range = FileRange.createFileRange(m * SEEK_SIZE, READ_SIZE);
       ranges.add(range);
     }
     stream.readVectored(ranges, bufferChoice.allocate);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[hadoop] 03/05: HADOOP-18107 Adding scale test for vectored reads for large file (#4273)

Posted by mt...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mthakur pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit bb5a17b177151fcbe4fcdb63bc892dcf6498323c
Author: Mukund Thakur <mt...@cloudera.com>
AuthorDate: Thu Jun 2 03:35:54 2022 +0530

    HADOOP-18107 Adding scale test for vectored reads for large file (#4273)
    
    part of HADOOP-18103.
    
    Contributed By: Mukund Thakur
---
 .../contract/AbstractContractVectoredReadTest.java | 86 +++-------------------
 .../hadoop/fs/contract/ContractTestUtils.java      | 64 ++++++++++++++++
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   |  1 +
 .../org/apache/hadoop/fs/s3a/S3AReadOpContext.java |  1 -
 .../fs/s3a/impl/GetContentSummaryOperation.java    |  3 +-
 .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java    | 33 +++++++++
 6 files changed, 111 insertions(+), 77 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
index 756c3de85cc..e8c86b5dbbc 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java
@@ -43,7 +43,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.impl.FutureIOSupport;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDatasetEquals;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
 
 @RunWith(Parameterized.class)
 public abstract class AbstractContractVectoredReadTest extends AbstractFSContractTestBase {
@@ -53,8 +55,6 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
   public static final int DATASET_LEN = 64 * 1024;
   private static final byte[] DATASET = ContractTestUtils.dataset(DATASET_LEN, 'a', 32);
   protected static final String VECTORED_READ_FILE_NAME = "vectored_file.txt";
-  private static final String VECTORED_READ_FILE_1MB_NAME = "vectored_file_1M.txt";
-  private static final byte[] DATASET_MB = ContractTestUtils.dataset(1024 * 1024, 'a', 256);
 
   private final IntFunction<ByteBuffer> allocate;
 
@@ -77,8 +77,6 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
     Path path = path(VECTORED_READ_FILE_NAME);
     FileSystem fs = getFileSystem();
     createFile(fs, path, true, DATASET);
-    Path bigFile = path(VECTORED_READ_FILE_1MB_NAME);
-    createFile(fs, bigFile, true, DATASET_MB);
   }
 
   @Test
@@ -99,7 +97,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
       CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(completableFutures);
       combinedFuture.get();
 
-      validateVectoredReadResult(fileRanges);
+      validateVectoredReadResult(fileRanges, DATASET);
     }
   }
 
@@ -132,7 +130,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
     fileRanges.add(new FileRangeImpl(16 * 1024 + 101, 100));
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
-      validateVectoredReadResult(fileRanges);
+      validateVectoredReadResult(fileRanges, DATASET);
     }
   }
 
@@ -149,7 +147,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
     fileRanges.add(new FileRangeImpl(8*1024 - 101, 100));
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
-      validateVectoredReadResult(fileRanges);
+      validateVectoredReadResult(fileRanges, DATASET);
     }
   }
 
@@ -168,7 +166,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
     fileRanges.add(new FileRangeImpl(40*1024, 1024));
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
-      validateVectoredReadResult(fileRanges);
+      validateVectoredReadResult(fileRanges, DATASET);
     }
   }
 
@@ -184,24 +182,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
                     .build();
     try (FSDataInputStream in = builder.get()) {
       in.readVectored(fileRanges, allocate);
-      validateVectoredReadResult(fileRanges);
-    }
-  }
-
-  @Test
-  public void testVectoredRead1MBFile()  throws Exception {
-    FileSystem fs = getFileSystem();
-    List<FileRange> fileRanges = new ArrayList<>();
-    fileRanges.add(new FileRangeImpl(1293, 25837));
-    CompletableFuture<FSDataInputStream> builder =
-            fs.openFile(path(VECTORED_READ_FILE_1MB_NAME))
-            .build();
-    try (FSDataInputStream in = builder.get()) {
-      in.readVectored(fileRanges, allocate);
-      ByteBuffer vecRes = FutureIOSupport.awaitFuture(fileRanges.get(0).getData());
-      FileRange resRange = fileRanges.get(0);
-      assertDatasetEquals((int) resRange.getOffset(), "vecRead",
-              vecRes, resRange.getLength(), DATASET_MB);
+      validateVectoredReadResult(fileRanges, DATASET);
     }
   }
 
@@ -215,7 +196,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
     fileRanges.add(new FileRangeImpl(10, 980));
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
-      validateVectoredReadResult(fileRanges);
+      validateVectoredReadResult(fileRanges, DATASET);
     }
   }
 
@@ -272,7 +253,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
       Assertions.assertThat(in.getPos())
               .describedAs("Vectored read shouldn't change file pointer.")
               .isEqualTo(200);
-      validateVectoredReadResult(fileRanges);
+      validateVectoredReadResult(fileRanges, DATASET);
     }
   }
 
@@ -290,7 +271,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
               .describedAs("Vectored read shouldn't change file pointer.")
               .isEqualTo(200);
       in.readVectored(fileRanges, allocate);
-      validateVectoredReadResult(fileRanges);
+      validateVectoredReadResult(fileRanges, DATASET);
     }
   }
 
@@ -302,8 +283,8 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges1, allocate);
       in.readVectored(fileRanges2, allocate);
-      validateVectoredReadResult(fileRanges2);
-      validateVectoredReadResult(fileRanges1);
+      validateVectoredReadResult(fileRanges2, DATASET);
+      validateVectoredReadResult(fileRanges1, DATASET);
     }
   }
 
@@ -314,27 +295,6 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
     return fileRanges;
   }
 
-  protected void validateVectoredReadResult(List<FileRange> fileRanges)
-          throws ExecutionException, InterruptedException {
-    CompletableFuture<?>[] completableFutures = new CompletableFuture<?>[fileRanges.size()];
-    int i = 0;
-    for (FileRange res : fileRanges) {
-      completableFutures[i++] = res.getData();
-    }
-    CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(completableFutures);
-    combinedFuture.get();
-
-    for (FileRange res : fileRanges) {
-      CompletableFuture<ByteBuffer> data = res.getData();
-      try {
-        ByteBuffer buffer = FutureIOSupport.awaitFuture(data);
-        assertDatasetEquals((int) res.getOffset(), "vecRead", buffer, res.getLength(), DATASET);
-      } catch (Exception ex) {
-        LOG.error("Exception while running vectored read ", ex);
-        Assert.fail("Exception while running vectored read " + ex);
-      }
-    }
-  }
 
   protected void testExceptionalVectoredRead(FileSystem fs,
                                              List<FileRange> fileRanges,
@@ -351,26 +311,4 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
             .describedAs(s)
             .isTrue();
   }
-
-  /**
-   * Assert that the data read matches the dataset at the given offset.
-   * This helps verify that the seek process is moving the read pointer
-   * to the correct location in the file.
-   *  @param readOffset the offset in the file where the read began.
-   * @param operation  operation name for the assertion.
-   * @param data       data read in.
-   * @param length     length of data to check.
-   * @param originalData
-   */
-  private void assertDatasetEquals(
-          final int readOffset, final String operation,
-          final ByteBuffer data,
-          int length, byte[] originalData) {
-    for (int i = 0; i < length; i++) {
-      int o = readOffset + i;
-      assertEquals(operation + " with read offset " + readOffset
-                      + ": data[" + i + "] != DATASET[" + o + "]",
-              originalData[o], data.get());
-    }
-  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
index 90e3649c1be..8c071282dc5 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.contract;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileRange;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -30,6 +31,7 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.hadoop.util.functional.FutureIO;
 
 import org.junit.Assert;
 import org.junit.AssumptionViolatedException;
@@ -41,6 +43,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -51,6 +54,9 @@ import java.util.NoSuchElementException;
 import java.util.Properties;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
@@ -70,6 +76,11 @@ public class ContractTestUtils extends Assert {
   public static final String IO_CHUNK_MODULUS_SIZE = "io.chunk.modulus.size";
   public static final int DEFAULT_IO_CHUNK_MODULUS_SIZE = 128;
 
+  /**
+   * Timeout in seconds for vectored read operation in tests : {@value}.
+   */
+  public static final int VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS = 5 * 60;
+
   /**
    * Assert that a property in the property set matches the expected value.
    * @param props property set
@@ -1097,6 +1108,59 @@ public class ContractTestUtils extends Assert {
                 mismatch);
   }
 
+  /**
+   * Utility to validate vectored read results.
+   * @param fileRanges input ranges.
+   * @param originalData original data.
+   * @throws IOException any ioe.
+   */
+  public static void validateVectoredReadResult(List<FileRange> fileRanges,
+                                                byte[] originalData)
+          throws IOException, TimeoutException {
+    CompletableFuture<?>[] completableFutures = new CompletableFuture<?>[fileRanges.size()];
+    int i = 0;
+    for (FileRange res : fileRanges) {
+      completableFutures[i++] = res.getData();
+    }
+    CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(completableFutures);
+    FutureIO.awaitFuture(combinedFuture,
+            VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS,
+            TimeUnit.SECONDS);
+
+    for (FileRange res : fileRanges) {
+      CompletableFuture<ByteBuffer> data = res.getData();
+      ByteBuffer buffer = FutureIO.awaitFuture(data,
+              VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS,
+              TimeUnit.SECONDS);
+      assertDatasetEquals((int) res.getOffset(), "vecRead",
+              buffer, res.getLength(), originalData);
+    }
+  }
+
+
+  /**
+   * Assert that the data read matches the dataset at the given offset.
+   * This helps verify that the seek process is moving the read pointer
+   * to the correct location in the file.
+   *  @param readOffset the offset in the file where the read began.
+   * @param operation  operation name for the assertion.
+   * @param data       data read in.
+   * @param length     length of data to check.
+   * @param originalData original data.
+   */
+  public static void assertDatasetEquals(
+          final int readOffset,
+          final String operation,
+          final ByteBuffer data,
+          int length, byte[] originalData) {
+    for (int i = 0; i < length; i++) {
+      int o = readOffset + i;
+      assertEquals(operation + " with read offset " + readOffset
+                      + ": data[" + i + "] != DATASET[" + o + "]",
+              originalData[o], data.get());
+    }
+  }
+
   /**
    * Receives test data from the given input file and checks the size of the
    * data as well as the pattern inside the received data.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index 23f31df1645..9d87f26c3c0 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -186,6 +186,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * @param ctx operation context
    * @param s3Attributes object attributes
    * @param client S3 client to use
+   * @param streamStatistics stream io stats.
    * @param unboundedThreadPool thread pool to use.
    */
   public S3AInputStream(S3AReadOpContext ctx,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
index 47de1d4d7d7..bb46ab898ba 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java
@@ -153,7 +153,6 @@ public class S3AReadOpContext extends S3AOpContext {
   }
 
   /**
-<<<<<<< HEAD
    * Set builder value.
    * @param value new value
    * @return the builder
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java
index 248bffb9401..257cef8192b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java
@@ -220,8 +220,7 @@ public class GetContentSummaryOperation extends
 
     /***
      * List all entries under a path.
-     *
-     * @param path
+     * @param path path.
      * @param recursive if the subdirectories need to be traversed recursively
      * @return an iterator over the listing.
      * @throws IOException failure
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
index 15700ce9535..956e23a3f11 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
@@ -19,8 +19,13 @@
 package org.apache.hadoop.fs.s3a.scale;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.IntFunction;
 
 import com.amazonaws.event.ProgressEvent;
 import com.amazonaws.event.ProgressEventType;
@@ -35,7 +40,10 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.FileRangeImpl;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.Constants;
@@ -47,6 +55,7 @@ import org.apache.hadoop.fs.statistics.IOStatistics;
 import org.apache.hadoop.util.Progressable;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
 import static org.apache.hadoop.fs.s3a.Statistic.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING;
@@ -446,6 +455,30 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
         toHuman(timer.nanosPerOperation(ops)));
   }
 
+  @Test
+  public void test_045_vectoredIOHugeFile() throws Throwable {
+    assumeHugeFileExists();
+    List<FileRange> rangeList = new ArrayList<>();
+    rangeList.add(new FileRangeImpl(5856368, 1167716));
+    rangeList.add(new FileRangeImpl(3520861, 1167700));
+    rangeList.add(new FileRangeImpl(8191913, 1167775));
+    rangeList.add(new FileRangeImpl(1520861, 1167700));
+    rangeList.add(new FileRangeImpl(2520861, 116770));
+    rangeList.add(new FileRangeImpl(9191913, 116770));
+    rangeList.add(new FileRangeImpl(2820861, 156770));
+    IntFunction<ByteBuffer> allocate = ByteBuffer::allocate;
+    FileSystem fs = getFileSystem();
+    CompletableFuture<FSDataInputStream> builder =
+            fs.openFile(hugefile).build();
+    try (FSDataInputStream in = builder.get()) {
+      in.readVectored(rangeList, allocate);
+      byte[] readFullRes = new byte[(int)filesize];
+      in.readFully(0, readFullRes);
+      // Comparing vectored read results with read fully.
+      validateVectoredReadResult(rangeList, readFullRes);
+    }
+  }
+
   /**
    * Read in the entire file using read() calls.
    * @throws Throwable failure


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[hadoop] 04/05: HADOOP-18105 Implement buffer pooling with weak references (#4263)

Posted by mt...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mthakur pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit bfb7d020d10eb232e8efebc7567e4999abcb5df4
Author: Mukund Thakur <mt...@cloudera.com>
AuthorDate: Thu Jun 2 03:38:06 2022 +0530

    HADOOP-18105 Implement buffer pooling with weak references (#4263)
    
    part of HADOOP-18103.
    Required for vectored IO feature. None of current buffer pool
    implementation is complete. ElasticByteBufferPool doesn't use
    weak references and could lead to memory leak errors and
    DirectBufferPool doesn't support caller preferences of direct
    and heap buffers and has only fixed length buffer implementation.
    
    Contributed By: Mukund Thakur
---
 .../java/org/apache/hadoop/io/ByteBufferPool.java  |   5 +
 .../apache/hadoop/io/ElasticByteBufferPool.java    |   4 +-
 .../io/WeakReferencedElasticByteBufferPool.java    | 155 ++++++++++++++
 ...estMoreWeakReferencedElasticByteBufferPool.java |  97 +++++++++
 .../TestWeakReferencedElasticByteBufferPool.java   | 232 +++++++++++++++++++++
 5 files changed, 491 insertions(+), 2 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java
index aa5f8731c54..b30e7cfb9c5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java
@@ -45,4 +45,9 @@ public interface ByteBufferPool {
    * @param buffer    a direct bytebuffer
    */
   void putBuffer(ByteBuffer buffer);
+
+  /**
+   * Clear the buffer pool thus releasing all the buffers.
+   */
+  default void release() { }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ElasticByteBufferPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ElasticByteBufferPool.java
index 6a162c3ff20..c4c29406227 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ElasticByteBufferPool.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ElasticByteBufferPool.java
@@ -36,8 +36,8 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public final class ElasticByteBufferPool implements ByteBufferPool {
-  private static final class Key implements Comparable<Key> {
+public class ElasticByteBufferPool implements ByteBufferPool {
+  protected static final class Key implements Comparable<Key> {
     private final int capacity;
     private final long insertionTime;
 
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java
new file mode 100644
index 00000000000..c71c44e798a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.lang.ref.WeakReference;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.VisibleForTesting;
+
+/**
+ * Buffer pool implementation which uses weak references to store
+ * buffers in the pool, such that they are garbage collected when
+ * there are no references to the buffer during a gc run. This is
+ * important as direct buffers don't get garbage collected automatically
+ * during a gc run as they are not stored on heap memory.
+ * Also the buffers are stored in a tree map which helps in returning
+ * smallest buffer whose size is just greater than requested length.
+ * This is a thread safe implementation.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class WeakReferencedElasticByteBufferPool extends ElasticByteBufferPool {
+
+  /**
+   * Map to store direct byte buffers of different sizes in the pool.
+   * Used tree map such that we can return next greater than capacity
+   * buffer if buffer with exact capacity is unavailable.
+   * This must be accessed in synchronized blocks.
+   */
+  private final TreeMap<Key, WeakReference<ByteBuffer>> directBuffers =
+          new TreeMap<>();
+
+  /**
+   * Map to store heap based byte buffers of different sizes in the pool.
+   * Used tree map such that we can return next greater than capacity
+   * buffer if buffer with exact capacity is unavailable.
+   * This must be accessed in synchronized blocks.
+   */
+  private final TreeMap<Key, WeakReference<ByteBuffer>> heapBuffers =
+          new TreeMap<>();
+
+  /**
+   * Method to get desired buffer tree.
+   * @param isDirect whether the buffer is heap based or direct.
+   * @return corresponding buffer tree.
+   */
+  private TreeMap<Key, WeakReference<ByteBuffer>> getBufferTree(boolean isDirect) {
+    return isDirect
+            ? directBuffers
+            : heapBuffers;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @param direct whether we want a direct byte buffer or a heap one.
+   * @param length length of requested buffer.
+   * @return returns equal or next greater than capacity buffer from
+   * pool if already available and not garbage collected else creates
+   * a new buffer and return it.
+   */
+  @Override
+  public synchronized ByteBuffer getBuffer(boolean direct, int length) {
+    TreeMap<Key, WeakReference<ByteBuffer>> buffersTree = getBufferTree(direct);
+
+    // Scan the entire tree and remove all weak null references.
+    buffersTree.entrySet().removeIf(next -> next.getValue().get() == null);
+
+    Map.Entry<Key, WeakReference<ByteBuffer>> entry =
+            buffersTree.ceilingEntry(new Key(length, 0));
+    // If there is no buffer present in the pool with desired size.
+    if (entry == null) {
+      return direct ? ByteBuffer.allocateDirect(length) :
+                      ByteBuffer.allocate(length);
+    }
+    // buffer is available in the pool and not garbage collected.
+    WeakReference<ByteBuffer> bufferInPool = entry.getValue();
+    buffersTree.remove(entry.getKey());
+    ByteBuffer buffer = bufferInPool.get();
+    if (buffer != null) {
+      return buffer;
+    }
+    // buffer was in pool but already got garbage collected.
+    return direct
+            ? ByteBuffer.allocateDirect(length)
+            : ByteBuffer.allocate(length);
+  }
+
+  /**
+   * Return buffer to the pool.
+   * @param buffer buffer to be returned.
+   */
+  @Override
+  public synchronized void putBuffer(ByteBuffer buffer) {
+    buffer.clear();
+    TreeMap<Key, WeakReference<ByteBuffer>> buffersTree = getBufferTree(buffer.isDirect());
+    // Buffers are indexed by (capacity, time).
+    // If our key is not unique on the first try, we try again, since the
+    // time will be different.  Since we use nanoseconds, it's pretty
+    // unlikely that we'll loop even once, unless the system clock has a
+    // poor granularity or multi-socket systems have clocks slightly out
+    // of sync.
+    while (true) {
+      Key keyToInsert = new Key(buffer.capacity(), System.nanoTime());
+      if (!buffersTree.containsKey(keyToInsert)) {
+        buffersTree.put(keyToInsert, new WeakReference<>(buffer));
+        return;
+      }
+    }
+  }
+
+  /**
+   * Clear the buffer pool thus releasing all the buffers.
+   * The caller must remove all references of
+   * existing buffers before calling this method to avoid
+   * memory leaks.
+   */
+  @Override
+  public synchronized void release() {
+    heapBuffers.clear();
+    directBuffers.clear();
+  }
+
+  /**
+   * Get current buffers count in the pool.
+   * @param isDirect whether we want to count the heap or direct buffers.
+   * @return count of buffers.
+   */
+  @VisibleForTesting
+  public synchronized int getCurrentBuffersCount(boolean isDirect) {
+    return isDirect
+            ? directBuffers.size()
+            : heapBuffers.size();
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMoreWeakReferencedElasticByteBufferPool.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMoreWeakReferencedElasticByteBufferPool.java
new file mode 100644
index 00000000000..6ca380ef0e4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMoreWeakReferencedElasticByteBufferPool.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.test.HadoopTestBase;
+
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Non parameterized tests for {@code WeakReferencedElasticByteBufferPool}.
+ */
+public class TestMoreWeakReferencedElasticByteBufferPool
+        extends HadoopTestBase {
+
+  @Test
+  public void testMixedBuffersInPool() {
+    WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
+    ByteBuffer buffer1 = pool.getBuffer(true, 5);
+    ByteBuffer buffer2 = pool.getBuffer(true, 10);
+    ByteBuffer buffer3 = pool.getBuffer(false, 5);
+    ByteBuffer buffer4 = pool.getBuffer(false, 10);
+    ByteBuffer buffer5 = pool.getBuffer(true, 15);
+
+    assertBufferCounts(pool, 0, 0);
+    pool.putBuffer(buffer1);
+    pool.putBuffer(buffer2);
+    assertBufferCounts(pool, 2, 0);
+    pool.putBuffer(buffer3);
+    assertBufferCounts(pool, 2, 1);
+    pool.putBuffer(buffer5);
+    assertBufferCounts(pool, 3, 1);
+    pool.putBuffer(buffer4);
+    assertBufferCounts(pool, 3, 2);
+    pool.release();
+    assertBufferCounts(pool, 0, 0);
+
+  }
+
+  @Test
+  public void testUnexpectedBufferSizes() throws Exception {
+    WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
+    ByteBuffer buffer1 = pool.getBuffer(true, 0);
+
+    // try writing a random byte in a 0 length buffer.
+    // Expected exception as buffer requested is of size 0.
+    intercept(BufferOverflowException.class,
+        () -> buffer1.put(new byte[1]));
+
+    // Expected IllegalArgumentException as negative length buffer is requested.
+    intercept(IllegalArgumentException.class,
+        () -> pool.getBuffer(true, -5));
+
+    // test returning null buffer to the pool.
+    intercept(NullPointerException.class,
+        () -> pool.putBuffer(null));
+  }
+
+  /**
+   * Utility method to assert counts of direct and heap buffers in
+   * the given buffer pool.
+   * @param pool buffer pool.
+   * @param numDirectBuffersExpected expected number of direct buffers.
+   * @param numHeapBuffersExpected expected number of heap buffers.
+   */
+  private void assertBufferCounts(WeakReferencedElasticByteBufferPool pool,
+                                  int numDirectBuffersExpected,
+                                  int numHeapBuffersExpected) {
+    Assertions.assertThat(pool.getCurrentBuffersCount(true))
+            .describedAs("Number of direct buffers in pool")
+            .isEqualTo(numDirectBuffersExpected);
+    Assertions.assertThat(pool.getCurrentBuffersCount(false))
+            .describedAs("Number of heap buffers in pool")
+            .isEqualTo(numHeapBuffersExpected);
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWeakReferencedElasticByteBufferPool.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWeakReferencedElasticByteBufferPool.java
new file mode 100644
index 00000000000..1434010ffa6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWeakReferencedElasticByteBufferPool.java
@@ -0,0 +1,232 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.test.HadoopTestBase;
+
+/**
+ * Unit tests for {@code WeakReferencedElasticByteBufferPool}.
+ */
+@RunWith(Parameterized.class)
+public class TestWeakReferencedElasticByteBufferPool
+        extends HadoopTestBase {
+
+  private final boolean isDirect;
+
+  private final String type;
+
+  @Parameterized.Parameters(name = "Buffer type : {0}")
+  public static List<String> params() {
+    return Arrays.asList("direct", "array");
+  }
+
+  public TestWeakReferencedElasticByteBufferPool(String type) {
+    this.type = type;
+    this.isDirect = !"array".equals(type);
+  }
+
+  @Test
+  public void testGetAndPutBasic() {
+    WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
+    int bufferSize = 5;
+    ByteBuffer buffer = pool.getBuffer(isDirect, bufferSize);
+    Assertions.assertThat(buffer.isDirect())
+            .describedAs("Buffered returned should be of correct type {}", type)
+            .isEqualTo(isDirect);
+    Assertions.assertThat(buffer.capacity())
+            .describedAs("Initial capacity of returned buffer from pool")
+            .isEqualTo(bufferSize);
+    Assertions.assertThat(buffer.position())
+            .describedAs("Initial position of returned buffer from pool")
+            .isEqualTo(0);
+
+    byte[] arr = createByteArray(bufferSize);
+    buffer.put(arr, 0, arr.length);
+    buffer.flip();
+    validateBufferContent(buffer, arr);
+    Assertions.assertThat(buffer.position())
+            .describedAs("Buffer's position after filling bytes in it")
+            .isEqualTo(bufferSize);
+    // releasing buffer to the pool.
+    pool.putBuffer(buffer);
+    Assertions.assertThat(buffer.position())
+            .describedAs("Position should be reset to 0 after returning buffer to the pool")
+            .isEqualTo(0);
+
+  }
+
+  @Test
+  public void testPoolingWithDifferentSizes() {
+    WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
+    ByteBuffer buffer = pool.getBuffer(isDirect, 5);
+    ByteBuffer buffer1 = pool.getBuffer(isDirect, 10);
+    ByteBuffer buffer2 = pool.getBuffer(isDirect, 15);
+
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool")
+            .isEqualTo(0);
+
+    pool.putBuffer(buffer1);
+    pool.putBuffer(buffer2);
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool")
+            .isEqualTo(2);
+    ByteBuffer buffer3 = pool.getBuffer(isDirect, 12);
+    Assertions.assertThat(buffer3.capacity())
+            .describedAs("Pooled buffer should have older capacity")
+            .isEqualTo(15);
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool")
+            .isEqualTo(1);
+    pool.putBuffer(buffer);
+    ByteBuffer buffer4 = pool.getBuffer(isDirect, 6);
+    Assertions.assertThat(buffer4.capacity())
+            .describedAs("Pooled buffer should have older capacity")
+            .isEqualTo(10);
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool")
+            .isEqualTo(1);
+
+    pool.release();
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool post release")
+            .isEqualTo(0);
+  }
+
+  @Test
+  public void testPoolingWithDifferentInsertionTime() {
+    WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
+    ByteBuffer buffer = pool.getBuffer(isDirect, 10);
+    ByteBuffer buffer1 = pool.getBuffer(isDirect, 10);
+    ByteBuffer buffer2 = pool.getBuffer(isDirect, 10);
+
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool")
+            .isEqualTo(0);
+
+    pool.putBuffer(buffer1);
+    pool.putBuffer(buffer2);
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool")
+            .isEqualTo(2);
+    ByteBuffer buffer3 = pool.getBuffer(isDirect, 10);
+    // As buffer1 is returned to the pool before buffer2, it should
+    // be returned when buffer of same size is asked again from
+    // the pool. Memory references must match not just content
+    // that is why {@code Assertions.isSameAs} is used here rather
+    // than usual {@code Assertions.isEqualTo}.
+    Assertions.assertThat(buffer3)
+            .describedAs("Buffers should be returned in order of their " +
+                    "insertion time")
+            .isSameAs(buffer1);
+    pool.putBuffer(buffer);
+    ByteBuffer buffer4 = pool.getBuffer(isDirect, 10);
+    Assertions.assertThat(buffer4)
+            .describedAs("Buffers should be returned in order of their " +
+                    "insertion time")
+            .isSameAs(buffer2);
+  }
+
+  @Test
+  public void testGarbageCollection() {
+    WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
+    ByteBuffer buffer = pool.getBuffer(isDirect, 5);
+    ByteBuffer buffer1 = pool.getBuffer(isDirect, 10);
+    ByteBuffer buffer2 = pool.getBuffer(isDirect, 15);
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool")
+            .isEqualTo(0);
+    pool.putBuffer(buffer1);
+    pool.putBuffer(buffer2);
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool")
+            .isEqualTo(2);
+    // Before GC.
+    ByteBuffer buffer4 = pool.getBuffer(isDirect, 12);
+    Assertions.assertThat(buffer4.capacity())
+            .describedAs("Pooled buffer should have older capacity")
+            .isEqualTo(15);
+    pool.putBuffer(buffer4);
+    // Removing the references
+    buffer1 = null;
+    buffer2 = null;
+    buffer4 = null;
+    System.gc();
+    ByteBuffer buffer3 = pool.getBuffer(isDirect, 12);
+    Assertions.assertThat(buffer3.capacity())
+            .describedAs("After garbage collection new buffer should be " +
+                    "returned with fixed capacity")
+            .isEqualTo(12);
+  }
+
+  @Test
+  public void testWeakReferencesPruning() {
+    WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
+    ByteBuffer buffer1 = pool.getBuffer(isDirect, 5);
+    ByteBuffer buffer2 = pool.getBuffer(isDirect, 10);
+    ByteBuffer buffer3 = pool.getBuffer(isDirect, 15);
+
+    pool.putBuffer(buffer2);
+    pool.putBuffer(buffer3);
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool")
+            .isEqualTo(2);
+
+    // marking only buffer2 to be garbage collected.
+    buffer2 = null;
+    System.gc();
+    ByteBuffer buffer4 = pool.getBuffer(isDirect, 10);
+    // Number of buffers in the pool is 0 as one got garbage
+    // collected and other got returned in above call.
+    Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
+            .describedAs("Number of buffers in the pool")
+            .isEqualTo(0);
+    Assertions.assertThat(buffer4.capacity())
+            .describedAs("After gc, pool should return next greater than " +
+                    "available buffer")
+            .isEqualTo(15);
+
+  }
+
+  private void validateBufferContent(ByteBuffer buffer, byte[] arr) {
+    for (int i=0; i<arr.length; i++) {
+      Assertions.assertThat(buffer.get())
+              .describedAs("Content of buffer at index {} should match " +
+                      "with content of byte array", i)
+              .isEqualTo(arr[i]);
+    }
+  }
+
+  private byte[] createByteArray(int length) {
+    byte[] arr = new byte[length];
+    Random r = new Random();
+    r.nextBytes(arr);
+    return arr;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org