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 st...@apache.org on 2022/04/24 16:36:16 UTC

[hadoop] branch trunk updated (17d64ba495e -> 44ae2fa8e53)

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

stevel pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


    from 17d64ba495e Revert "HDFS-16488. [SPS]: Expose metrics to JMX for external SPS (#4035)" (#4232)
     new 1b4dba99b5d HADOOP-16202. Enhanced openFile(): hadoop-common changes. (#2584/1)
     new 6999acf5200 HADOOP-16202. Enhanced openFile(): mapreduce and YARN changes. (#2584/2)
     new e0cd0a82e03 HADOOP-16202. Enhanced openFile(): hadoop-aws changes. (#2584/3)
     new 44ae2fa8e53 HADOOP-16202. Enhanced openFile(): hadoop-azure changes. (#2584/4)

The 4 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:
 .../java/org/apache/hadoop/fs/AvroFSInput.java     |  11 +-
 .../org/apache/hadoop/fs/ChecksumFileSystem.java   |   4 +-
 .../main/java/org/apache/hadoop/fs/FSBuilder.java  |  14 +
 .../java/org/apache/hadoop/fs/FileContext.java     |  18 +-
 .../main/java/org/apache/hadoop/fs/FileSystem.java |  11 +-
 .../main/java/org/apache/hadoop/fs/FileUtil.java   |  61 ++-
 .../hadoop/fs/FutureDataInputStreamBuilder.java    |   8 +-
 .../main/java/org/apache/hadoop/fs/Options.java    | 119 ++++
 .../hadoop/fs/impl/AbstractFSBuilderImpl.java      |  38 +-
 .../fs/impl/FileSystemMultipartUploader.java       |   9 +-
 .../fs/impl/FutureDataInputStreamBuilderImpl.java  |   8 +-
 .../org/apache/hadoop/fs/impl/FutureIOSupport.java |  83 +--
 .../apache/hadoop/fs/impl/OpenFileParameters.java  |  13 +
 .../apache/hadoop/fs/impl/WrappedIOException.java  |  11 +-
 .../hadoop/fs/shell/CommandWithDestination.java    |   9 +-
 .../org/apache/hadoop/fs/shell/CopyCommands.java   |   3 +-
 .../java/org/apache/hadoop/fs/shell/Display.java   |   3 +-
 .../main/java/org/apache/hadoop/fs/shell/Head.java |   8 +-
 .../java/org/apache/hadoop/fs/shell/PathData.java  |  35 ++
 .../main/java/org/apache/hadoop/fs/shell/Tail.java |  11 +-
 .../hadoop/fs/statistics/StoreStatisticNames.java  |   9 +
 .../hadoop/fs/statistics/StreamStatisticNames.java |  19 +-
 .../fs/statistics/impl/IOStatisticsBinding.java    |  44 +-
 .../java/org/apache/hadoop/io/SequenceFile.java    |  14 +-
 .../org/apache/hadoop/util/JsonSerialization.java  |   6 +-
 .../util/functional/CommonCallableSupplier.java    |   2 +-
 .../apache/hadoop/util/functional/FutureIO.java    |  90 ++++
 .../src/site/markdown/filesystem/filesystem.md     |  90 +---
 .../filesystem/fsdatainputstreambuilder.md         | 588 +++++++++++++++++++-
 .../src/site/markdown/filesystem/index.md          |   1 +
 .../src/site/markdown/filesystem/openfile.md       | 122 +++++
 .../AbstractContractMultipartUploaderTest.java     |   2 +-
 .../fs/contract/AbstractContractOpenTest.java      |  80 ++-
 .../hadoop/fs/contract/ContractTestUtils.java      |  13 +-
 .../fs/statistics/IOStatisticAssertions.java       |  20 +
 .../hadoop/fs/statistics/TestDurationTracking.java |   3 +-
 .../jobhistory/JobHistoryCopyService.java          |  10 +-
 .../org/apache/hadoop/mapred/LineRecordReader.java |  13 +-
 .../lib/input/FixedLengthRecordReader.java         |   7 +-
 .../mapreduce/lib/input/LineRecordReader.java      |  14 +-
 .../mapreduce/lib/input/NLineInputFormat.java      |   6 +-
 .../hadoop/examples/terasort/TeraInputFormat.java  |  16 +-
 .../hadoop-aws/dev-support/findbugs-exclude.xml    |   5 +
 .../java/org/apache/hadoop/fs/s3a/Constants.java   |  45 +-
 .../java/org/apache/hadoop/fs/s3a/Invoker.java     |  28 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java    | 283 +++++-----
 .../org/apache/hadoop/fs/s3a/S3AInputPolicy.java   |  93 +++-
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   | 263 ++++++---
 .../apache/hadoop/fs/s3a/S3AInstrumentation.java   |  12 +-
 .../org/apache/hadoop/fs/s3a/S3AReadOpContext.java | 101 +++-
 .../apache/hadoop/fs/s3a/S3ObjectAttributes.java   |  15 +-
 .../java/org/apache/hadoop/fs/s3a/Statistic.java   |  17 +
 .../hadoop/fs/s3a/commit/CommitOperations.java     |   8 +-
 .../hadoop/fs/s3a/commit/files/PendingSet.java     |  26 +-
 .../fs/s3a/commit/files/SinglePendingCommit.java   |  20 +-
 .../hadoop/fs/s3a/impl/AbstractStoreOperation.java |  26 +-
 .../hadoop/fs/s3a/impl/CallableSupplier.java       |   2 +-
 .../hadoop/fs/s3a/impl/InternalConstants.java      |  21 +-
 .../apache/hadoop/fs/s3a/impl/OpenFileSupport.java | 600 +++++++++++++++++++++
 .../apache/hadoop/fs/s3a/s3guard/S3GuardTool.java  |   4 +-
 .../fs/s3a/select/InternalSelectConstants.java     |   2 +-
 .../apache/hadoop/fs/s3a/select/SelectTool.java    |   4 +-
 .../s3a/statistics/S3AInputStreamStatistics.java   |   7 +
 .../statistics/impl/EmptyS3AStatisticsContext.java |   4 +
 .../src/site/markdown/tools/hadoop-aws/index.md    |   8 +
 .../fs/contract/s3a/ITestS3AContractOpen.java      |  67 +++
 .../fs/contract/s3a/ITestS3AContractSeek.java      |  15 +-
 .../apache/hadoop/fs/s3a/AbstractS3AMockTest.java  |   4 +
 .../hadoop/fs/s3a/ITestS3AConfiguration.java       |  11 -
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java     |   4 +-
 .../hadoop/fs/s3a/TestS3AInputStreamRetry.java     |  17 +-
 .../org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java  |   5 +-
 .../hadoop/fs/s3a/TestStreamChangeTracker.java     |   2 +-
 .../hadoop/fs/s3a/impl/TestOpenFileSupport.java    | 429 +++++++++++++++
 .../fs/s3a/performance/ITestS3AOpenCost.java       | 209 +++++++
 .../hadoop/fs/s3a/performance/OperationCost.java   |   6 +
 .../s3a/scale/ITestS3AInputStreamPerformance.java  |  58 +-
 .../hadoop/fs/s3a/select/AbstractS3SelectTest.java |   2 +-
 .../apache/hadoop/fs/s3a/select/ITestS3Select.java |   7 +-
 .../hadoop/fs/s3a/select/ITestS3SelectMRJob.java   |   4 +-
 .../fs/azurebfs/AzureBlobFileSystemStore.java      |   2 +-
 .../contract/ITestAbfsFileSystemContractSeek.java  |   2 +-
 .../tools/mapred/RetriableFileCopyCommand.java     |   9 +-
 .../streaming/mapreduce/StreamInputFormat.java     |   6 +-
 .../yarn/logaggregation/AggregatedLogFormat.java   |  17 +-
 .../org/apache/hadoop/yarn/util/FSDownload.java    |  24 +-
 86 files changed, 3478 insertions(+), 662 deletions(-)
 create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
 create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java
 create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileSupport.java
 create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java


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


[hadoop] 01/04: HADOOP-16202. Enhanced openFile(): hadoop-common changes. (#2584/1)

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

stevel pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 1b4dba99b5d941eb8fd4787462a998c13ea2f171
Author: Steve Loughran <st...@cloudera.com>
AuthorDate: Sun Apr 24 17:03:59 2022 +0100

    HADOOP-16202. Enhanced openFile(): hadoop-common changes. (#2584/1)
    
    This defines standard option and values for the
    openFile() builder API for opening a file:
    
    fs.option.openfile.read.policy
     A list of the desired read policy, in preferred order.
     standard values are
     adaptive, default, random, sequential, vector, whole-file
    
    fs.option.openfile.length
     How long the file is.
    
    fs.option.openfile.split.start
     start of a task's split
    
    fs.option.openfile.split.end
     end of a task's split
    
    These can be used by filesystem connectors to optimize their
    reading of the source file, including but not limited to
    * skipping existence/length probes when opening a file
    * choosing a policy for prefetching/caching data
    
    The hadoop shell commands which read files all declare "whole-file"
    and "sequential", as appropriate.
    
    Contributed by Steve Loughran.
    
    Change-Id: Ia290f79ea7973ce8713d4f90f1315b24d7a23da1
---
 .../java/org/apache/hadoop/fs/AvroFSInput.java     |  11 +-
 .../org/apache/hadoop/fs/ChecksumFileSystem.java   |   4 +-
 .../main/java/org/apache/hadoop/fs/FSBuilder.java  |  14 +
 .../java/org/apache/hadoop/fs/FileContext.java     |  18 +-
 .../main/java/org/apache/hadoop/fs/FileSystem.java |  11 +-
 .../main/java/org/apache/hadoop/fs/FileUtil.java   |  61 ++-
 .../hadoop/fs/FutureDataInputStreamBuilder.java    |   8 +-
 .../main/java/org/apache/hadoop/fs/Options.java    | 119 +++++
 .../hadoop/fs/impl/AbstractFSBuilderImpl.java      |  38 +-
 .../fs/impl/FileSystemMultipartUploader.java       |   9 +-
 .../fs/impl/FutureDataInputStreamBuilderImpl.java  |   8 +-
 .../org/apache/hadoop/fs/impl/FutureIOSupport.java |  83 +--
 .../apache/hadoop/fs/impl/OpenFileParameters.java  |  13 +
 .../apache/hadoop/fs/impl/WrappedIOException.java  |  11 +-
 .../hadoop/fs/shell/CommandWithDestination.java    |   9 +-
 .../org/apache/hadoop/fs/shell/CopyCommands.java   |   3 +-
 .../java/org/apache/hadoop/fs/shell/Display.java   |   3 +-
 .../main/java/org/apache/hadoop/fs/shell/Head.java |   8 +-
 .../java/org/apache/hadoop/fs/shell/PathData.java  |  35 ++
 .../main/java/org/apache/hadoop/fs/shell/Tail.java |  11 +-
 .../hadoop/fs/statistics/StoreStatisticNames.java  |   9 +
 .../hadoop/fs/statistics/StreamStatisticNames.java |  19 +-
 .../fs/statistics/impl/IOStatisticsBinding.java    |  44 +-
 .../java/org/apache/hadoop/io/SequenceFile.java    |  14 +-
 .../org/apache/hadoop/util/JsonSerialization.java  |   6 +-
 .../util/functional/CommonCallableSupplier.java    |   2 +-
 .../apache/hadoop/util/functional/FutureIO.java    |  90 ++++
 .../src/site/markdown/filesystem/filesystem.md     |  90 +---
 .../filesystem/fsdatainputstreambuilder.md         | 588 ++++++++++++++++++++-
 .../src/site/markdown/filesystem/index.md          |   1 +
 .../src/site/markdown/filesystem/openfile.md       | 122 +++++
 .../AbstractContractMultipartUploaderTest.java     |   2 +-
 .../fs/contract/AbstractContractOpenTest.java      |  80 ++-
 .../hadoop/fs/contract/ContractTestUtils.java      |  13 +-
 .../fs/statistics/IOStatisticAssertions.java       |  20 +
 .../hadoop/fs/statistics/TestDurationTracking.java |   3 +-
 36 files changed, 1320 insertions(+), 260 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java
index b4a4a85674d..213fbc24c4d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java
@@ -25,6 +25,10 @@ import org.apache.avro.file.SeekableInput;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
+
 /** Adapts an {@link FSDataInputStream} to Avro's SeekableInput interface. */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
@@ -42,7 +46,12 @@ public class AvroFSInput implements Closeable, SeekableInput {
   public AvroFSInput(final FileContext fc, final Path p) throws IOException {
     FileStatus status = fc.getFileStatus(p);
     this.len = status.getLen();
-    this.stream = fc.open(p);
+    this.stream = awaitFuture(fc.openFile(p)
+        .opt(FS_OPTION_OPENFILE_READ_POLICY,
+            FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL)
+        .withFileStatus(status)
+        .build());
+    fc.open(p);
   }
 
   @Override
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 c7f8e36c3f6..59ffe00bcb2 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
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.nio.channels.ClosedChannelException;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
@@ -45,6 +44,7 @@ import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.LambdaUtils;
 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;
 
@@ -889,7 +889,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       final OpenFileParameters parameters) throws IOException {
     AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
         parameters.getMandatoryKeys(),
-        Collections.emptySet(),
+        FS_OPTION_OPENFILE_STANDARD_OPTIONS,
         "for " + path);
     return LambdaUtils.eval(
         new CompletableFuture<>(),
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java
index b7757a62e28..a4c7254cfeb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java
@@ -61,6 +61,13 @@ public interface FSBuilder<S, B extends FSBuilder<S, B>> {
    */
   B opt(@Nonnull String key, float value);
 
+  /**
+   * Set optional long parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  B opt(@Nonnull String key, long value);
+
   /**
    * Set optional double parameter for the Builder.
    *
@@ -104,6 +111,13 @@ public interface FSBuilder<S, B extends FSBuilder<S, B>> {
    */
   B must(@Nonnull String key, float value);
 
+  /**
+   * Set mandatory long option.
+   *
+   * @see #must(String, String)
+   */
+  B must(@Nonnull String key, long value);
+
   /**
    * Set mandatory double option.
    *
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
index 9922dfa0ac8..f3004ce7e03 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
@@ -70,7 +70,12 @@ import org.apache.hadoop.tracing.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /**
  * The FileContext class provides an interface for users of the Hadoop
@@ -2198,7 +2203,12 @@ public class FileContext implements PathCapabilities {
         EnumSet<CreateFlag> createFlag = overwrite ? EnumSet.of(
             CreateFlag.CREATE, CreateFlag.OVERWRITE) :
             EnumSet.of(CreateFlag.CREATE);
-        InputStream in = open(qSrc);
+        InputStream in = awaitFuture(openFile(qSrc)
+            .opt(FS_OPTION_OPENFILE_READ_POLICY,
+                FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
+            .opt(FS_OPTION_OPENFILE_LENGTH,
+                fs.getLen())   // file length hint for object stores
+            .build());
         try (OutputStream out = create(qDst, createFlag)) {
           IOUtils.copyBytes(in, out, conf, true);
         } finally {
@@ -2930,9 +2940,11 @@ public class FileContext implements PathCapabilities {
       final Path absF = fixRelativePart(getPath());
       OpenFileParameters parameters = new OpenFileParameters()
           .withMandatoryKeys(getMandatoryKeys())
+          .withOptionalKeys(getOptionalKeys())
           .withOptions(getOptions())
-          .withBufferSize(getBufferSize())
-          .withStatus(getStatus());
+          .withStatus(getStatus())
+          .withBufferSize(
+              getOptions().getInt(FS_OPTION_OPENFILE_BUFFER_SIZE, getBufferSize()));
       return new FSLinkResolver<CompletableFuture<FSDataInputStream>>() {
         @Override
         public CompletableFuture<FSDataInputStream> next(
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index fdb1a475520..aa194e84a35 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -88,6 +88,7 @@ import org.apache.hadoop.classification.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE;
 import static org.apache.hadoop.util.Preconditions.checkArgument;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
@@ -4616,7 +4617,7 @@ public abstract class FileSystem extends Configured
       final OpenFileParameters parameters) throws IOException {
     AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
         parameters.getMandatoryKeys(),
-        Collections.emptySet(),
+        Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS,
         "for " + path);
     return LambdaUtils.eval(
         new CompletableFuture<>(), () ->
@@ -4644,7 +4645,7 @@ public abstract class FileSystem extends Configured
       final OpenFileParameters parameters) throws IOException {
     AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
         parameters.getMandatoryKeys(),
-        Collections.emptySet(), "");
+        Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS, "");
     CompletableFuture<FSDataInputStream> result = new CompletableFuture<>();
     try {
       result.complete(open(pathHandle, parameters.getBufferSize()));
@@ -4751,9 +4752,11 @@ public abstract class FileSystem extends Configured
       Optional<Path> optionalPath = getOptionalPath();
       OpenFileParameters parameters = new OpenFileParameters()
           .withMandatoryKeys(getMandatoryKeys())
+          .withOptionalKeys(getOptionalKeys())
           .withOptions(getOptions())
-          .withBufferSize(getBufferSize())
-          .withStatus(super.getStatus());  // explicit to avoid IDE warnings
+          .withStatus(super.getStatus())
+          .withBufferSize(
+              getOptions().getInt(FS_OPTION_OPENFILE_BUFFER_SIZE, getBufferSize()));
       if(optionalPath.isPresent()) {
         return getFS().openFileWithOptions(optionalPath.get(),
             parameters);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
index 0d5ced79fc4..7400ca36daa 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
@@ -77,6 +77,11 @@ import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
+
 /**
  * A collection of file-processing util methods
  */
@@ -396,7 +401,32 @@ public class FileUtil {
     return copy(srcFS, fileStatus, dstFS, dst, deleteSource, overwrite, conf);
   }
 
-  /** Copy files between FileSystems. */
+  /**
+   * Copy a file/directory tree within/between filesystems.
+   * <p></p>
+   * returns true if the operation succeeded. When deleteSource is true,
+   * this means "after the copy, delete(source) returned true"
+   * If the destination is a directory, and mkdirs (dest) fails,
+   * the operation will return false rather than raise any exception.
+   * <p></p>
+   * The overwrite flag is about overwriting files; it has no effect about
+   * handing an attempt to copy a file atop a directory (expect an IOException),
+   * or a directory over a path which contains a file (mkdir will fail, so
+   * "false").
+   * <p></p>
+   * The operation is recursive, and the deleteSource operation takes place
+   * as each subdirectory is copied. Therefore, if an operation fails partway
+   * through, the source tree may be partially deleted.
+   * @param srcFS source filesystem
+   * @param srcStatus status of source
+   * @param dstFS destination filesystem
+   * @param dst path of source
+   * @param deleteSource delete the source?
+   * @param overwrite overwrite files at destination?
+   * @param conf configuration to use when opening files
+   * @return true if the operation succeeded.
+   * @throws IOException failure
+   */
   public static boolean copy(FileSystem srcFS, FileStatus srcStatus,
                              FileSystem dstFS, Path dst,
                              boolean deleteSource,
@@ -409,22 +439,27 @@ public class FileUtil {
       if (!dstFS.mkdirs(dst)) {
         return false;
       }
-      FileStatus contents[] = srcFS.listStatus(src);
-      for (int i = 0; i < contents.length; i++) {
-        copy(srcFS, contents[i], dstFS,
-             new Path(dst, contents[i].getPath().getName()),
-             deleteSource, overwrite, conf);
+      RemoteIterator<FileStatus> contents = srcFS.listStatusIterator(src);
+      while (contents.hasNext()) {
+        FileStatus next = contents.next();
+        copy(srcFS, next, dstFS,
+            new Path(dst, next.getPath().getName()),
+            deleteSource, overwrite, conf);
       }
     } else {
-      InputStream in=null;
+      InputStream in = null;
       OutputStream out = null;
       try {
-        in = srcFS.open(src);
+        in = awaitFuture(srcFS.openFile(src)
+            .opt(FS_OPTION_OPENFILE_READ_POLICY,
+                FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
+            .opt(FS_OPTION_OPENFILE_LENGTH,
+                srcStatus.getLen())   // file length hint for object stores
+            .build());
         out = dstFS.create(dst, overwrite);
         IOUtils.copyBytes(in, out, conf, true);
       } catch (IOException e) {
-        IOUtils.closeStream(out);
-        IOUtils.closeStream(in);
+        IOUtils.cleanupWithLogger(LOG, in, out);
         throw e;
       }
     }
@@ -503,7 +538,11 @@ public class FileUtil {
              deleteSource, conf);
       }
     } else {
-      InputStream in = srcFS.open(src);
+      InputStream in = awaitFuture(srcFS.openFile(src)
+          .withFileStatus(srcStatus)
+          .opt(FS_OPTION_OPENFILE_READ_POLICY,
+              FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
+          .build());
       IOUtils.copyBytes(in, Files.newOutputStream(dst.toPath()), conf);
     }
     if (deleteSource) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java
index 27a522e5930..e7f441a75d3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.fs;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 
@@ -34,7 +35,7 @@ import org.apache.hadoop.classification.InterfaceStability;
  * options accordingly, for example:
  *
  * If the option is not related to the file system, the option will be ignored.
- * If the option is must, but not supported by the file system, a
+ * If the option is must, but not supported/known by the file system, an
  * {@link IllegalArgumentException} will be thrown.
  *
  */
@@ -51,10 +52,11 @@ public interface FutureDataInputStreamBuilder
   /**
    * A FileStatus may be provided to the open request.
    * It is up to the implementation whether to use this or not.
-   * @param status status.
+   * @param status status: may be null
    * @return the builder.
    */
-  default FutureDataInputStreamBuilder withFileStatus(FileStatus status) {
+  default FutureDataInputStreamBuilder withFileStatus(
+      @Nullable FileStatus status) {
     return this;
   }
 
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
index 75bc12df8fd..9b457272fcb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
@@ -17,9 +17,13 @@
  */
 package org.apache.hadoop.fs;
 
+import java.util.Collections;
 import java.util.Optional;
+import java.util.Set;
 import java.util.function.Function;
 import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -518,4 +522,119 @@ public final class Options {
     MD5MD5CRC,  // MD5 of block checksums, which are MD5 over chunk CRCs
     COMPOSITE_CRC  // Block/chunk-independent composite CRC
   }
+
+  /**
+   * The standard {@code openFile()} options.
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public static final class OpenFileOptions {
+
+    private OpenFileOptions() {
+    }
+
+    /**
+     * Prefix for all standard filesystem options: {@value}.
+     */
+    private static final String FILESYSTEM_OPTION = "fs.option.";
+
+    /**
+     * Prefix for all openFile options: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE =
+        FILESYSTEM_OPTION + "openfile.";
+
+    /**
+     * OpenFile option for file length: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_LENGTH =
+        FS_OPTION_OPENFILE + "length";
+
+    /**
+     * OpenFile option for split start: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_SPLIT_START =
+        FS_OPTION_OPENFILE + "split.start";
+
+    /**
+     * OpenFile option for split end: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_SPLIT_END =
+        FS_OPTION_OPENFILE + "split.end";
+
+    /**
+     * OpenFile option for buffer size: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_BUFFER_SIZE =
+        FS_OPTION_OPENFILE + "buffer.size";
+
+    /**
+     * OpenFile option for read policies: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY =
+        FS_OPTION_OPENFILE + "read.policy";
+
+    /**
+     * Set of standard options which openFile implementations
+     * MUST recognize, even if they ignore the actual values.
+     */
+    public static final Set<String> FS_OPTION_OPENFILE_STANDARD_OPTIONS =
+        Collections.unmodifiableSet(Stream.of(
+                FS_OPTION_OPENFILE_BUFFER_SIZE,
+                FS_OPTION_OPENFILE_READ_POLICY,
+                FS_OPTION_OPENFILE_LENGTH,
+                FS_OPTION_OPENFILE_SPLIT_START,
+                FS_OPTION_OPENFILE_SPLIT_END)
+            .collect(Collectors.toSet()));
+
+    /**
+     * Read policy for adaptive IO: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE =
+        "adaptive";
+
+    /**
+     * Read policy {@value} -whateve the implementation does by default.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY_DEFAULT =
+        "default";
+
+    /**
+     * Read policy for random IO: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY_RANDOM =
+        "random";
+
+    /**
+     * Read policy for sequential IO: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL =
+        "sequential";
+
+    /**
+     * Vectored IO API to be used: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY_VECTOR =
+        "vector";
+
+    /**
+     * Whole file to be read, end-to-end: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE =
+        "whole-file";
+
+    /**
+     * All the current read policies as a set.
+     */
+    public static final Set<String> FS_OPTION_OPENFILE_READ_POLICIES =
+        Collections.unmodifiableSet(Stream.of(
+                FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE,
+                FS_OPTION_OPENFILE_READ_POLICY_DEFAULT,
+                FS_OPTION_OPENFILE_READ_POLICY_RANDOM,
+                FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL,
+                FS_OPTION_OPENFILE_READ_POLICY_VECTOR,
+                FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
+            .collect(Collectors.toSet()));
+
+  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java
index c69e7afe4e3..9d3a46d6332 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java
@@ -46,7 +46,7 @@ import static org.apache.hadoop.util.Preconditions.checkNotNull;
  * <code>
  *   .opt("foofs:option.a", true)
  *   .opt("foofs:option.b", "value")
- *   .opt("barfs:cache", true)
+ *   .opt("fs.s3a.open.option.etag", "9fe4c37c25b")
  *   .must("foofs:cache", true)
  *   .must("barfs:cache-size", 256 * 1024 * 1024)
  *   .build();
@@ -88,6 +88,9 @@ public abstract class
   /** Keep track of the keys for mandatory options. */
   private final Set<String> mandatoryKeys = new HashSet<>();
 
+  /** Keep track of the optional keys. */
+  private final Set<String> optionalKeys = new HashSet<>();
+
   /**
    * Constructor with both optional path and path handle.
    * Either or both argument may be empty, but it is an error for
@@ -163,6 +166,7 @@ public abstract class
   @Override
   public B opt(@Nonnull final String key, @Nonnull final String value) {
     mandatoryKeys.remove(key);
+    optionalKeys.add(key);
     options.set(key, value);
     return getThisBuilder();
   }
@@ -175,6 +179,7 @@ public abstract class
   @Override
   public B opt(@Nonnull final String key, boolean value) {
     mandatoryKeys.remove(key);
+    optionalKeys.add(key);
     options.setBoolean(key, value);
     return getThisBuilder();
   }
@@ -187,10 +192,19 @@ public abstract class
   @Override
   public B opt(@Nonnull final String key, int value) {
     mandatoryKeys.remove(key);
+    optionalKeys.add(key);
     options.setInt(key, value);
     return getThisBuilder();
   }
 
+  @Override
+  public B opt(@Nonnull final String key, final long value) {
+    mandatoryKeys.remove(key);
+    optionalKeys.add(key);
+    options.setLong(key, value);
+    return getThisBuilder();
+  }
+
   /**
    * Set optional float parameter for the Builder.
    *
@@ -199,6 +213,7 @@ public abstract class
   @Override
   public B opt(@Nonnull final String key, float value) {
     mandatoryKeys.remove(key);
+    optionalKeys.add(key);
     options.setFloat(key, value);
     return getThisBuilder();
   }
@@ -211,6 +226,7 @@ public abstract class
   @Override
   public B opt(@Nonnull final String key, double value) {
     mandatoryKeys.remove(key);
+    optionalKeys.add(key);
     options.setDouble(key, value);
     return getThisBuilder();
   }
@@ -223,6 +239,7 @@ public abstract class
   @Override
   public B opt(@Nonnull final String key, @Nonnull final String... values) {
     mandatoryKeys.remove(key);
+    optionalKeys.add(key);
     options.setStrings(key, values);
     return getThisBuilder();
   }
@@ -248,6 +265,7 @@ public abstract class
   @Override
   public B must(@Nonnull final String key, boolean value) {
     mandatoryKeys.add(key);
+    optionalKeys.remove(key);
     options.setBoolean(key, value);
     return getThisBuilder();
   }
@@ -260,10 +278,19 @@ public abstract class
   @Override
   public B must(@Nonnull final String key, int value) {
     mandatoryKeys.add(key);
+    optionalKeys.remove(key);
     options.setInt(key, value);
     return getThisBuilder();
   }
 
+  @Override
+  public B must(@Nonnull final String key, final long value) {
+    mandatoryKeys.add(key);
+    optionalKeys.remove(key);
+    options.setLong(key, value);
+    return getThisBuilder();
+  }
+
   /**
    * Set mandatory float option.
    *
@@ -272,6 +299,7 @@ public abstract class
   @Override
   public B must(@Nonnull final String key, float value) {
     mandatoryKeys.add(key);
+    optionalKeys.remove(key);
     options.setFloat(key, value);
     return getThisBuilder();
   }
@@ -284,6 +312,7 @@ public abstract class
   @Override
   public B must(@Nonnull final String key, double value) {
     mandatoryKeys.add(key);
+    optionalKeys.remove(key);
     options.setDouble(key, value);
     return getThisBuilder();
   }
@@ -296,6 +325,7 @@ public abstract class
   @Override
   public B must(@Nonnull final String key, @Nonnull final String... values) {
     mandatoryKeys.add(key);
+    optionalKeys.remove(key);
     options.setStrings(key, values);
     return getThisBuilder();
   }
@@ -314,6 +344,12 @@ public abstract class
   public Set<String> getMandatoryKeys() {
     return Collections.unmodifiableSet(mandatoryKeys);
   }
+  /**
+   * Get all the keys that are set as optional keys.
+   */
+  public Set<String> getOptionalKeys() {
+    return Collections.unmodifiableSet(optionalKeys);
+  }
 
   /**
    * Reject a configuration if one or more mandatory keys are
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java
index 2339e421289..481d927672d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.fs.UploadHandle;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.functional.FutureIO;
 
 import static org.apache.hadoop.fs.Path.mergePaths;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
@@ -98,7 +99,7 @@ public class FileSystemMultipartUploader extends AbstractMultipartUploader {
   public CompletableFuture<UploadHandle> startUpload(Path filePath)
       throws IOException {
     checkPath(filePath);
-    return FutureIOSupport.eval(() -> {
+    return FutureIO.eval(() -> {
       Path collectorPath = createCollectorPath(filePath);
       fs.mkdirs(collectorPath, FsPermission.getDirDefault());
 
@@ -116,7 +117,7 @@ public class FileSystemMultipartUploader extends AbstractMultipartUploader {
       throws IOException {
     checkPutArguments(filePath, inputStream, partNumber, uploadId,
         lengthInBytes);
-    return FutureIOSupport.eval(() -> innerPutPart(filePath,
+    return FutureIO.eval(() -> innerPutPart(filePath,
         inputStream, partNumber, uploadId, lengthInBytes));
   }
 
@@ -179,7 +180,7 @@ public class FileSystemMultipartUploader extends AbstractMultipartUploader {
       Map<Integer, PartHandle> handleMap) throws IOException {
 
     checkPath(filePath);
-    return FutureIOSupport.eval(() ->
+    return FutureIO.eval(() ->
         innerComplete(uploadId, filePath, handleMap));
   }
 
@@ -251,7 +252,7 @@ public class FileSystemMultipartUploader extends AbstractMultipartUploader {
     Path collectorPath = new Path(new String(uploadIdByteArray, 0,
         uploadIdByteArray.length, Charsets.UTF_8));
 
-    return FutureIOSupport.eval(() -> {
+    return FutureIO.eval(() -> {
       // force a check for a file existing; raises FNFE if not found
       fs.getFileStatus(collectorPath);
       fs.delete(collectorPath, true);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java
index 24a8d49747f..70e39de7388 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.impl;
 
 import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 
@@ -47,7 +48,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
  * options accordingly, for example:
  *
  * If the option is not related to the file system, the option will be ignored.
- * If the option is must, but not supported by the file system, a
+ * If the option is must, but not supported/known by the file system, an
  * {@link IllegalArgumentException} will be thrown.
  *
  */
@@ -147,8 +148,9 @@ public abstract class FutureDataInputStreamBuilderImpl
   }
 
   @Override
-  public FutureDataInputStreamBuilder withFileStatus(FileStatus st) {
-    this.status = requireNonNull(st, "status");
+  public FutureDataInputStreamBuilder withFileStatus(
+      @Nullable FileStatus st) {
+    this.status = st;
     return this;
   }
 
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java
index 18f5187cb61..f47e5f4fbfb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.impl;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
-import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionException;
 import java.util.concurrent.ExecutionException;
@@ -37,14 +36,16 @@ import org.apache.hadoop.util.functional.FutureIO;
 
 /**
  * Support for future IO and the FS Builder subclasses.
- * If methods in here are needed for applications, promote
- * to {@link FutureIO} for public use -with the original
- * method relaying to it. This is to ensure that external
- * filesystem implementations can safely use these methods
+ * All methods in this class have been superceded by those in
+ * {@link FutureIO}.
+ * The methods here are retained but all marked as deprecated.
+ * This is to ensure that any external
+ * filesystem implementations can still use these methods
  * without linkage problems surfacing.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
+@Deprecated
 public final class FutureIOSupport {
 
   private FutureIOSupport() {
@@ -53,6 +54,7 @@ public final class FutureIOSupport {
   /**
    * Given a future, evaluate it. Raised exceptions are
    * extracted and handled.
+   * See {@link FutureIO#awaitFuture(Future, long, TimeUnit)}.
    * @param future future to evaluate
    * @param <T> type of the result.
    * @return the result, if all went well.
@@ -60,7 +62,8 @@ public final class FutureIOSupport {
    * @throws IOException if something went wrong
    * @throws RuntimeException any nested RTE thrown
    */
-  public static <T> T  awaitFuture(final Future<T> future)
+  @Deprecated
+  public static <T> T awaitFuture(final Future<T> future)
       throws InterruptedIOException, IOException, RuntimeException {
     return FutureIO.awaitFuture(future);
   }
@@ -69,6 +72,7 @@ public final class FutureIOSupport {
   /**
    * Given a future, evaluate it. Raised exceptions are
    * extracted and handled.
+   * See {@link FutureIO#awaitFuture(Future, long, TimeUnit)}.
    * @param future future to evaluate
    * @param <T> type of the result.
    * @return the result, if all went well.
@@ -77,6 +81,7 @@ public final class FutureIOSupport {
    * @throws RuntimeException any nested RTE thrown
    * @throws TimeoutException the future timed out.
    */
+  @Deprecated
   public static <T> T awaitFuture(final Future<T> future,
       final long timeout,
       final TimeUnit unit)
@@ -88,10 +93,7 @@ public final class FutureIOSupport {
   /**
    * From the inner cause of an execution exception, extract the inner cause
    * if it is an IOE or RTE.
-   * This will always raise an exception, either the inner IOException,
-   * an inner RuntimeException, or a new IOException wrapping the raised
-   * exception.
-   *
+   * See {@link FutureIO#raiseInnerCause(ExecutionException)}.
    * @param e exception.
    * @param <T> type of return value.
    * @return nothing, ever.
@@ -99,6 +101,7 @@ public final class FutureIOSupport {
    * any non-Runtime-Exception
    * @throws RuntimeException if that is the inner cause.
    */
+  @Deprecated
   public static <T> T raiseInnerCause(final ExecutionException e)
       throws IOException {
     return FutureIO.raiseInnerCause(e);
@@ -107,6 +110,7 @@ public final class FutureIOSupport {
   /**
    * Extract the cause of a completion failure and rethrow it if an IOE
    * or RTE.
+   * See {@link FutureIO#raiseInnerCause(CompletionException)}.
    * @param e exception.
    * @param <T> type of return value.
    * @return nothing, ever.
@@ -114,20 +118,15 @@ public final class FutureIOSupport {
    * any non-Runtime-Exception
    * @throws RuntimeException if that is the inner cause.
    */
+  @Deprecated
   public static <T> T raiseInnerCause(final CompletionException e)
       throws IOException {
     return FutureIO.raiseInnerCause(e);
   }
 
   /**
-   * Propagate options to any builder, converting everything with the
-   * prefix to an option where, if there were 2+ dot-separated elements,
-   * it is converted to a schema.
-   * <pre>{@code
-   *   fs.example.s3a.option => s3a:option
-   *   fs.example.fs.io.policy => s3a.io.policy
-   *   fs.example.something => something
-   * }</pre>
+   * Propagate options to any builder.
+   * {@link FutureIO#propagateOptions(FSBuilder, Configuration, String, String)}
    * @param builder builder to modify
    * @param conf configuration to read
    * @param optionalPrefix prefix for optional settings
@@ -136,56 +135,39 @@ public final class FutureIOSupport {
    * @param <U> type of builder
    * @return the builder passed in.
    */
+  @Deprecated
   public static <T, U extends FSBuilder<T, U>>
         FSBuilder<T, U> propagateOptions(
       final FSBuilder<T, U> builder,
       final Configuration conf,
       final String optionalPrefix,
       final String mandatoryPrefix) {
-    propagateOptions(builder, conf,
-        optionalPrefix, false);
-    propagateOptions(builder, conf,
-        mandatoryPrefix, true);
-    return builder;
+    return FutureIO.propagateOptions(builder,
+        conf, optionalPrefix, mandatoryPrefix);
   }
 
   /**
-   * Propagate options to any builder, converting everything with the
-   * prefix to an option where, if there were 2+ dot-separated elements,
-   * it is converted to a schema.
-   * <pre>{@code
-   *   fs.example.s3a.option => s3a:option
-   *   fs.example.fs.io.policy => s3a.io.policy
-   *   fs.example.something => something
-   * }</pre>
+   * Propagate options to any builder.
+   * {@link FutureIO#propagateOptions(FSBuilder, Configuration, String, boolean)}
    * @param builder builder to modify
    * @param conf configuration to read
    * @param prefix prefix to scan/strip
    * @param mandatory are the options to be mandatory or optional?
    */
+  @Deprecated
   public static void propagateOptions(
       final FSBuilder<?, ?> builder,
       final Configuration conf,
       final String prefix,
       final boolean mandatory) {
-
-    final String p = prefix.endsWith(".") ? prefix : (prefix + ".");
-    final Map<String, String> propsWithPrefix = conf.getPropsWithPrefix(p);
-    for (Map.Entry<String, String> entry : propsWithPrefix.entrySet()) {
-      // change the schema off each entry
-      String key = entry.getKey();
-      String val = entry.getValue();
-      if (mandatory) {
-        builder.must(key, val);
-      } else {
-        builder.opt(key, val);
-      }
-    }
+    FutureIO.propagateOptions(builder, conf, prefix, mandatory);
   }
 
   /**
    * Evaluate a CallableRaisingIOE in the current thread,
    * converting IOEs to RTEs and propagating.
+   * See {@link FutureIO#eval(CallableRaisingIOE)}.
+   *
    * @param callable callable to invoke
    * @param <T> Return type.
    * @return the evaluated result.
@@ -194,17 +176,6 @@ public final class FutureIOSupport {
    */
   public static <T> CompletableFuture<T> eval(
       CallableRaisingIOE<T> callable) {
-    CompletableFuture<T> result = new CompletableFuture<>();
-    try {
-      result.complete(callable.apply());
-    } catch (UnsupportedOperationException | IllegalArgumentException tx) {
-      // fail fast here
-      throw tx;
-    } catch (Throwable tx) {
-      // fail lazily here to ensure callers expect all File IO operations to
-      // surface later
-      result.completeExceptionally(tx);
-    }
-    return result;
+    return FutureIO.eval(callable);
   }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java
index 77b4ff52696..a19c5faff4d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java
@@ -38,6 +38,9 @@ public class OpenFileParameters {
    */
   private Set<String> mandatoryKeys;
 
+  /** The optional keys. */
+  private Set<String> optionalKeys;
+
   /**
    * Options set during the build sequence.
    */
@@ -61,6 +64,11 @@ public class OpenFileParameters {
     return this;
   }
 
+  public OpenFileParameters withOptionalKeys(final Set<String> keys) {
+    this.optionalKeys = requireNonNull(keys);
+    return this;
+  }
+
   public OpenFileParameters withOptions(final Configuration opts) {
     this.options = requireNonNull(opts);
     return this;
@@ -80,6 +88,10 @@ public class OpenFileParameters {
     return mandatoryKeys;
   }
 
+  public Set<String> getOptionalKeys() {
+    return optionalKeys;
+  }
+
   public Configuration getOptions() {
     return options;
   }
@@ -91,4 +103,5 @@ public class OpenFileParameters {
   public FileStatus getStatus() {
     return status;
   }
+
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java
index d2c999683c6..d5144b5e9c5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java
@@ -20,20 +20,17 @@ package org.apache.hadoop.fs.impl;
 
 import java.io.IOException;
 import java.io.UncheckedIOException;
-import java.util.concurrent.ExecutionException;
 
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * A wrapper for an IOException which
- * {@link FutureIOSupport#raiseInnerCause(ExecutionException)} knows to
- * always extract the exception.
+ * A wrapper for an IOException.
  *
  * The constructor signature guarantees the cause will be an IOException,
  * and as it checks for a null-argument, non-null.
- * @deprecated use the {@code UncheckedIOException}.
+ * @deprecated use the {@code UncheckedIOException} directly.]
  */
 @Deprecated
 @InterfaceAudience.Private
@@ -51,8 +48,4 @@ public class WrappedIOException extends UncheckedIOException {
     super(Preconditions.checkNotNull(cause));
   }
 
-  @Override
-  public synchronized IOException getCause() {
-    return (IOException) super.getCause();
-  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
index f6f4247489f..678225f81e0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
@@ -55,6 +55,9 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
 import static org.apache.hadoop.fs.CreateFlag.CREATE;
 import static org.apache.hadoop.fs.CreateFlag.LAZY_PERSIST;
 import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /**
  * Provides: argument processing to ensure the destination is valid
@@ -348,7 +351,11 @@ abstract class CommandWithDestination extends FsCommand {
     src.fs.setVerifyChecksum(verifyChecksum);
     InputStream in = null;
     try {
-      in = src.fs.open(src.path);
+      in = awaitFuture(src.fs.openFile(src.path)
+          .withFileStatus(src.stat)
+          .opt(FS_OPTION_OPENFILE_READ_POLICY,
+              FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
+          .build());
       copyStreamToTarget(in, target);
       preserveAttributes(src, target, preserveRawXattrs);
     } finally {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
index b03d7de8a1c..0643a2e983d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
@@ -98,7 +98,8 @@ class CopyCommands {
       try {
         for (PathData src : srcs) {
           if (src.stat.getLen() != 0) {
-            try (FSDataInputStream in = src.fs.open(src.path)) {
+            // Always do sequential reads.
+            try (FSDataInputStream in = src.openForSequentialIO()) {
               IOUtils.copyBytes(in, out, getConf(), false);
               writeDelimiter(out);
             }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java
index 670fa152f72..d3ca013a3f2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java
@@ -105,7 +105,8 @@ class Display extends FsCommand {
     }
 
     protected InputStream getInputStream(PathData item) throws IOException {
-      return item.fs.open(item.path);
+      // Always do sequential reads;
+      return item.openForSequentialIO();
     }
   }
   
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Head.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Head.java
index 2280225b5ae..7242f261801 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Head.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Head.java
@@ -28,6 +28,8 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+
 /**
  * Show the first 1KB of the file.
  */
@@ -68,11 +70,9 @@ class Head extends FsCommand {
   }
 
   private void dumpToOffset(PathData item) throws IOException {
-    FSDataInputStream in = item.fs.open(item.path);
-    try {
+    try (FSDataInputStream in = item.openFile(
+        FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL)) {
       IOUtils.copyBytes(in, System.out, endingOffset, false);
-    } finally {
-      in.close();
     }
   }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
index 1ff8d8f0494..2071a16799a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
@@ -29,6 +29,7 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -39,6 +40,10 @@ import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.PathNotFoundException;
 import org.apache.hadoop.fs.RemoteIterator;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator;
 
 /**
@@ -601,4 +606,34 @@ public class PathData implements Comparable<PathData> {
   public int hashCode() {
     return path.hashCode();
   }
+
+
+  /**
+   * Open a file for sequential IO.
+   * <p></p>
+   * This uses FileSystem.openFile() to request sequential IO;
+   * the file status is also passed in.
+   * Filesystems may use to optimize their IO.
+   * @return an input stream
+   * @throws IOException failure
+   */
+  protected FSDataInputStream openForSequentialIO()
+      throws IOException {
+    return openFile(FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL);
+  }
+
+  /**
+   * Open a file.
+   * @param policy fadvise policy.
+   * @return an input stream
+   * @throws IOException failure
+   */
+  protected FSDataInputStream openFile(final String policy) throws IOException {
+    return awaitFuture(fs.openFile(path)
+        .opt(FS_OPTION_OPENFILE_READ_POLICY,
+            policy)
+        .opt(FS_OPTION_OPENFILE_LENGTH,
+            stat.getLen())   // file length hint for object stores
+        .build());
+  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java
index 98db1c3b7b6..26ac3fee471 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.io.IOUtils;
 
 import org.apache.hadoop.classification.VisibleForTesting;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+
 /**
  * Get a listing of all files in that match the file patterns.
  */
@@ -107,16 +109,15 @@ class Tail extends FsCommand {
     if (offset < 0) {
       offset = Math.max(fileSize + offset, 0);
     }
-    
-    FSDataInputStream in = item.fs.open(item.path);
-    try {
+    // Always do sequential reads.
+    try (FSDataInputStream in = item.openFile(
+        FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL)) {
       in.seek(offset);
       // use conf so the system configured io block size is used
       IOUtils.copyBytes(in, System.out, getConf(), false);
       offset = in.getPos();
-    } finally {
-      in.close();
     }
     return offset;
   }
+
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
index 166007f5c9a..c458269c351 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java
@@ -118,6 +118,9 @@ public final class StoreStatisticNames {
   /** {@value}. */
   public static final String OP_OPEN = "op_open";
 
+  /** Call to openFile() {@value}. */
+  public static final String OP_OPENFILE = "op_openfile";
+
   /** {@value}. */
   public static final String OP_REMOVE_ACL = "op_remove_acl";
 
@@ -323,6 +326,12 @@ public final class StoreStatisticNames {
   public static final String ACTION_EXECUTOR_ACQUIRED =
       "action_executor_acquired";
 
+  /**
+   * A file was opened: {@value}.
+   */
+  public static final String ACTION_FILE_OPENED
+      = "action_file_opened";
+
   /**
    * An HTTP HEAD request was made: {@value}.
    */
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java
index 7e9137294c1..ca755f08419 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java
@@ -76,7 +76,7 @@ public final class StreamStatisticNames {
   public static final String STREAM_READ_CLOSED = "stream_read_closed";
 
   /**
-   * Total count of times an attempt to close an input stream was made
+   * Total count of times an attempt to close an input stream was made.
    * Value: {@value}.
    */
   public static final String STREAM_READ_CLOSE_OPERATIONS
@@ -118,6 +118,23 @@ public final class StreamStatisticNames {
   public static final String STREAM_READ_OPERATIONS_INCOMPLETE
       = "stream_read_operations_incomplete";
 
+  /**
+   * count/duration of aborting a remote stream during stream IO
+   * IO.
+   * Value: {@value}.
+   */
+  public static final String STREAM_READ_REMOTE_STREAM_ABORTED
+      = "stream_read_remote_stream_aborted";
+
+  /**
+   * count/duration of closing a remote stream,
+   * possibly including draining the stream to recycle
+   * the HTTP connection.
+   * Value: {@value}.
+   */
+  public static final String STREAM_READ_REMOTE_STREAM_DRAINED
+      = "stream_read_remote_stream_drain";
+
   /**
    * Count of version mismatches encountered while reading an input stream.
    * Value: {@value}.
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java
index 70366fc7a30..c45dfc21a1b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java
@@ -521,23 +521,39 @@ public final class IOStatisticsBinding {
       // create the tracker outside try-with-resources so
       // that failures can be set in the catcher.
       DurationTracker tracker = createTracker(factory, statistic);
-      try {
-        // exec the input function and return its value
-        return input.apply();
-      } catch (IOException | RuntimeException e) {
-        // input function failed: note it
-        tracker.failed();
-        // and rethrow
-        throw e;
-      } finally {
-        // update the tracker.
-        // this is called after the catch() call would have
-        // set the failed flag.
-        tracker.close();
-      }
+      return invokeTrackingDuration(tracker, input);
     };
   }
 
+  /**
+   * Given an IOException raising callable/lambda expression,
+   * execute it, updating the tracker on success/failure.
+   * @param tracker duration tracker.
+   * @param input input callable.
+   * @param <B> return type.
+   * @return the result of the invocation
+   * @throws IOException on failure.
+   */
+  public static <B> B invokeTrackingDuration(
+      final DurationTracker tracker,
+      final CallableRaisingIOE<B> input)
+      throws IOException {
+    try {
+      // exec the input function and return its value
+      return input.apply();
+    } catch (IOException | RuntimeException e) {
+      // input function failed: note it
+      tracker.failed();
+      // and rethrow
+      throw e;
+    } finally {
+      // update the tracker.
+      // this is called after the catch() call would have
+      // set the failed flag.
+      tracker.close();
+    }
+  }
+
   /**
    * Given an IOException raising Consumer,
    * return a new one which wraps the inner and tracks
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
index 037bbd3fd0f..890e7916ab0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
@@ -57,6 +57,11 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SEQFILE_COMP
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SEQFILE_COMPRESS_BLOCKSIZE_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SKIP_CHECKSUM_ERRORS_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SKIP_CHECKSUM_ERRORS_KEY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /** 
  * <code>SequenceFile</code>s are flat files consisting of binary key/value 
@@ -1948,7 +1953,14 @@ public class SequenceFile {
      */
     protected FSDataInputStream openFile(FileSystem fs, Path file,
         int bufferSize, long length) throws IOException {
-      return fs.open(file, bufferSize);
+      FutureDataInputStreamBuilder builder = fs.openFile(file)
+          .opt(FS_OPTION_OPENFILE_READ_POLICY,
+              FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL)
+          .opt(FS_OPTION_OPENFILE_BUFFER_SIZE, bufferSize);
+      if (length >= 0) {
+        builder.opt(FS_OPTION_OPENFILE_LENGTH, length);
+      }
+      return awaitFuture(builder.build());
     }
     
     /**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java
index d2bc8cd2960..002c725490f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java
@@ -49,6 +49,8 @@ import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIOException;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
 import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /**
@@ -265,7 +267,9 @@ public class JsonSerialization<T> {
     if (status != null && status.getLen() == 0) {
       throw new EOFException("No data in " + path);
     }
-    FutureDataInputStreamBuilder builder = fs.openFile(path);
+    FutureDataInputStreamBuilder builder = fs.openFile(path)
+        .opt(FS_OPTION_OPENFILE_READ_POLICY,
+            FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE);
     if (status != null) {
       builder.withFileStatus(status);
     }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java
index e2cdc0fd414..32e299b4d45 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java
@@ -34,7 +34,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.util.DurationInfo;
 
-import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause;
+import static org.apache.hadoop.util.functional.FutureIO.raiseInnerCause;
 
 /**
  * A bridge from Callable to Supplier; catching exceptions
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java
index 3f7218baa75..c3fda19d8d7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/FutureIO.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.util.functional;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.io.UncheckedIOException;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
@@ -29,6 +31,8 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSBuilder;
 
 /**
  * Future IO Helper methods.
@@ -86,6 +90,8 @@ public final class FutureIO {
    * extracted and rethrown.
    * </p>
    * @param future future to evaluate
+   * @param timeout timeout to wait
+   * @param unit time unit.
    * @param <T> type of the result.
    * @return the result, if all went well.
    * @throws InterruptedIOException future was interrupted
@@ -185,4 +191,88 @@ public final class FutureIO {
     }
   }
 
+  /**
+   * Propagate options to any builder, converting everything with the
+   * prefix to an option where, if there were 2+ dot-separated elements,
+   * it is converted to a schema.
+   * See {@link #propagateOptions(FSBuilder, Configuration, String, boolean)}.
+   * @param builder builder to modify
+   * @param conf configuration to read
+   * @param optionalPrefix prefix for optional settings
+   * @param mandatoryPrefix prefix for mandatory settings
+   * @param <T> type of result
+   * @param <U> type of builder
+   * @return the builder passed in.
+   */
+  public static <T, U extends FSBuilder<T, U>>
+      FSBuilder<T, U> propagateOptions(
+        final FSBuilder<T, U> builder,
+        final Configuration conf,
+        final String optionalPrefix,
+        final String mandatoryPrefix) {
+    propagateOptions(builder, conf,
+        optionalPrefix, false);
+    propagateOptions(builder, conf,
+        mandatoryPrefix, true);
+    return builder;
+  }
+
+  /**
+   * Propagate options to any builder, converting everything with the
+   * prefix to an option where, if there were 2+ dot-separated elements,
+   * it is converted to a schema.
+   * <pre>
+   *   fs.example.s3a.option becomes "s3a.option"
+   *   fs.example.fs.io.policy becomes "fs.io.policy"
+   *   fs.example.something becomes "something"
+   * </pre>
+   * @param builder builder to modify
+   * @param conf configuration to read
+   * @param prefix prefix to scan/strip
+   * @param mandatory are the options to be mandatory or optional?
+   */
+  public static void propagateOptions(
+      final FSBuilder<?, ?> builder,
+      final Configuration conf,
+      final String prefix,
+      final boolean mandatory) {
+
+    final String p = prefix.endsWith(".") ? prefix : (prefix + ".");
+    final Map<String, String> propsWithPrefix = conf.getPropsWithPrefix(p);
+    for (Map.Entry<String, String> entry : propsWithPrefix.entrySet()) {
+      // change the schema off each entry
+      String key = entry.getKey();
+      String val = entry.getValue();
+      if (mandatory) {
+        builder.must(key, val);
+      } else {
+        builder.opt(key, val);
+      }
+    }
+  }
+
+  /**
+   * Evaluate a CallableRaisingIOE in the current thread,
+   * converting IOEs to RTEs and propagating.
+   * @param callable callable to invoke
+   * @param <T> Return type.
+   * @return the evaluated result.
+   * @throws UnsupportedOperationException fail fast if unsupported
+   * @throws IllegalArgumentException invalid argument
+   */
+  public static <T> CompletableFuture<T> eval(
+      CallableRaisingIOE<T> callable) {
+    CompletableFuture<T> result = new CompletableFuture<>();
+    try {
+      result.complete(callable.apply());
+    } catch (UnsupportedOperationException | IllegalArgumentException tx) {
+      // fail fast here
+      throw tx;
+    } catch (Throwable tx) {
+      // fail lazily here to ensure callers expect all File IO operations to
+      // surface later
+      result.completeExceptionally(tx);
+    }
+    return result;
+  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
index 4517bd8ff4a..004220c4bed 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
@@ -814,97 +814,11 @@ exists in the metadata, but no copies of any its blocks can be located;
 
 ### `FSDataInputStreamBuilder openFile(Path path)`
 
-Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
-to construct a operation to open the file at `path` for reading.
+See [openFile()](openfile.html).
 
-When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
-the builder parameters are verified and
-`openFileWithOptions(Path, OpenFileParameters)` invoked.
-
-This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
-which, when its `get()` method is called, either returns an input
-stream of the contents of opened file, or raises an exception.
-
-The base implementation of the `openFileWithOptions(PathHandle, OpenFileParameters)`
-ultimately invokes `open(Path, int)`.
-
-Thus the chain `openFile(path).build().get()` has the same preconditions
-and postconditions as `open(Path p, int bufferSize)`
-
-However, there is one difference which implementations are free to
-take advantage of: 
-
-The returned stream MAY implement a lazy open where file non-existence or
-access permission failures may not surface until the first `read()` of the
-actual data.
-
-The `openFile()` operation may check the state of the filesystem during its
-invocation, but as the state of the filesystem may change betwen this call and
-the actual `build()` and `get()` operations, this file-specific
-preconditions (file exists, file is readable, etc) MUST NOT be checked here.
-
-FileSystem implementations which do not implement `open(Path, int)`
-MAY postpone raising an `UnsupportedOperationException` until either the
-`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
-else they MAY fail fast in the `openFile()` call.
-
-### Implementors notes
-
-The base implementation of `openFileWithOptions()` actually executes
-the `open(path)` operation synchronously, yet still returns the result
-or any failures in the `CompletableFuture<>`, so as to ensure that users
-code expecting this.
-
-Any filesystem where the time to open a file may be significant SHOULD
-execute it asynchronously by submitting the operation in some executor/thread
-pool. This is particularly recommended for object stores and other filesystems
-likely to be accessed over long-haul connections.
-
-Arbitrary filesystem-specific options MAY be supported; these MUST
-be prefixed with either the filesystem schema, e.g. `hdfs.`
-or in the "fs.SCHEMA" format as normal configuration settings `fs.hdfs`). The
-latter style allows the same configuration option to be used for both
-filesystem configuration and file-specific configuration.
-
-It SHOULD be possible to always open a file without specifying any options,
-so as to present a consistent model to users. However, an implementation MAY
-opt to require one or more mandatory options to be set.
-
-The returned stream may perform "lazy" evaluation of file access. This is
-relevant for object stores where the probes for existence are expensive, and,
-even with an asynchronous open, may be considered needless.
- 
 ### `FSDataInputStreamBuilder openFile(PathHandle)`
 
-Creates a `FSDataInputStreamBuilder` to build an operation to open a file.
-Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
-to construct a operation to open the file identified by the given `PathHandle` for reading.
-
-When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
-the builder parameters are verified and
-`openFileWithOptions(PathHandle, OpenFileParameters)` invoked.
-
-This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
-which, when its `get()` method is called, either returns an input
-stream of the contents of opened file, or raises an exception.
-
-The base implementation of the `openFileWithOptions(PathHandle, OpenFileParameters)` method
-returns a future which invokes `open(Path, int)`.
-
-Thus the chain `openFile(pathhandle).build().get()` has the same preconditions
-and postconditions as `open(Pathhandle, int)`
-
-As with `FSDataInputStreamBuilder openFile(PathHandle)`, the `openFile()`
-call must not be where path-specific preconditions are checked -that
-is postponed to the `build()` and `get()` calls.
-
-FileSystem implementations which do not implement `open(PathHandle handle, int bufferSize)`
-MAY postpone raising an `UnsupportedOperationException` until either the
-`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
-else they MAY fail fast in the `openFile()` call.
-
-The base implementation raises this exception in the `build()` operation;
-other implementations SHOULD copy this.
+See [openFile()](openfile.html).
 
 ### `PathHandle getPathHandle(FileStatus stat, HandleOpt... options)`
 
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md
index eadba174fc1..db630e05c22 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md
@@ -13,10 +13,10 @@
 -->
 
 <!--  ============================================================= -->
-<!--  CLASS: FSDataInputStreamBuilder -->
+<!--  CLASS: FutureDataInputStreamBuilder -->
 <!--  ============================================================= -->
 
-# class `org.apache.hadoop.fs.FSDataInputStreamBuilder`
+# class `org.apache.hadoop.fs.FutureDataInputStreamBuilder`
 
 <!-- MACRO{toc|fromDepth=1|toDepth=2} -->
 
@@ -27,7 +27,7 @@ file for reading.
 
 ## Invariants
 
-The `FSDataInputStreamBuilder` interface does not require parameters or
+The `FutureDataInputStreamBuilder` interface does not require parameters or
 or the state of `FileSystem` until [`build()`](#build) is
 invoked and/or during the asynchronous open operation itself.
 
@@ -39,11 +39,11 @@ path validation.
 ## Implementation-agnostic parameters.
 
 
-### <a name="Builder.bufferSize"></a> `FSDataInputStreamBuilder bufferSize(int bufSize)`
+### <a name="Builder.bufferSize"></a> `FutureDataInputStreamBuilder bufferSize(int bufSize)`
 
 Set the size of the buffer to be used.
 
-### <a name="Builder.withFileStatus"></a> `FSDataInputStreamBuilder withFileStatus(FileStatus status)`
+### <a name="Builder.withFileStatus"></a> `FutureDataInputStreamBuilder withFileStatus(FileStatus status)`
 
 A `FileStatus` instance which refers to the file being opened.
 
@@ -53,7 +53,7 @@ So potentially saving on remote calls especially to object stores.
 Requirements:
 
 * `status != null`
-* `status.getPath()` == the resolved path of the file being opened.
+* `status.getPath().getName()` == the name of the file being opened.
 
 The path validation MUST take place if the store uses the `FileStatus` when
 it opens files, and MAY be performed otherwise. The validation
@@ -65,27 +65,85 @@ If a filesystem implementation extends the `FileStatus` returned in its
 implementation MAY use this information when opening the file.
 
 This is relevant with those stores which return version/etag information,
-including the S3A and ABFS connectors -they MAY use this to guarantee that
-the file they opened is exactly the one returned in the listing.
+-they MAY use this to guarantee that the file they opened
+is exactly the one returned in the listing.
+
+
+The final `status.getPath().getName()` element of the supplied status MUST equal
+the name value of the path supplied to the  `openFile(path)` call.
+
+Filesystems MUST NOT validate the rest of the path.
+This is needed to support viewfs and other mount-point wrapper filesystems
+where schemas and paths are different. These often create their own FileStatus results
+
+Preconditions
+
+```python
+status == null or status.getPath().getName() == path.getName()
+
+```
+
+Filesystems MUST NOT require the class of `status` to equal
+that of any specific subclass their implementation returns in filestatus/list
+operations. This is to support wrapper filesystems and serialization/deserialization
+of the status.
+
 
 ### Set optional or mandatory parameters
 
-    FSDataInputStreamBuilder opt(String key, ...)
-    FSDataInputStreamBuilder must(String key, ...)
+    FutureDataInputStreamBuilder opt(String key, ...)
+    FutureDataInputStreamBuilder must(String key, ...)
 
 Set optional or mandatory parameters to the builder. Using `opt()` or `must()`,
 client can specify FS-specific parameters without inspecting the concrete type
 of `FileSystem`.
 
+Example:
+
 ```java
 out = fs.openFile(path)
-    .opt("fs.s3a.experimental.input.fadvise", "random")
-    .must("fs.s3a.readahead.range", 256 * 1024)
+    .must("fs.option.openfile.read.policy", "random")
+    .opt("fs.http.connection.timeout", 30_000L)
     .withFileStatus(statusFromListing)
     .build()
     .get();
 ```
 
+Here the read policy of `random` has been specified,
+with the requirement that the filesystem implementation must understand the option.
+An http-specific option has been supplied which may be interpreted by any store;
+If the filesystem opening the file does not recognize the option, it can safely be
+ignored.
+
+### When to use `opt()` versus `must()`
+
+The difference between `opt()` versus `must()` is how the FileSystem opening
+the file must react to an option which it does not recognize.
+
+```python
+
+def must(name, value):
+  if not name in known_keys:
+    raise IllegalArgumentException
+  if not name in supported_keys:
+    raise UnsupportedException
+
+
+def opt(name, value):
+  if not name in known_keys:
+     # ignore option
+
+```
+
+For any known key, the validation of the `value` argument MUST be the same
+irrespective of how the (key, value) pair was declared.
+
+1. For a filesystem-specific option, it is the choice of the implementation
+   how to validate the entry.
+1. For standard options, the specification of what is a valid `value` is
+   defined in this filesystem specification, validated through contract
+   tests.
+
 #### Implementation Notes
 
 Checking for supported options must be performed in the `build()` operation.
@@ -93,9 +151,9 @@ Checking for supported options must be performed in the `build()` operation.
 1. If a mandatory parameter declared via `must(key, value)`) is not recognized,
 `IllegalArgumentException` MUST be thrown.
 
-1. If a mandatory parameter declared via `must(key, value)`) relies on
+1. If a mandatory parameter declared via `must(key, value)` relies on
 a feature which is recognized but not supported in the specific
-Filesystem/FileContext instance `UnsupportedException` MUST be thrown.
+`FileSystem`/`FileContext` instance `UnsupportedException` MUST be thrown.
 
 The behavior of resolving the conflicts between the parameters set by
 builder methods (i.e., `bufferSize()`) and `opt()`/`must()` is as follows:
@@ -110,13 +168,18 @@ custom subclasses.
 
 This is critical to ensure safe use of the feature: directory listing/
 status serialization/deserialization can result result in the `withFileStatus()`
-argumennt not being the custom subclass returned by the Filesystem instance's
+argument not being the custom subclass returned by the Filesystem instance's
 own `getFileStatus()`, `listFiles()`, `listLocatedStatus()` calls, etc.
 
 In such a situation the implementations must:
 
-1. Validate the path (always).
-1. Use the status/convert to the custom type, *or* simply discard it.
+1. Verify that `status.getPath().getName()` matches the current `path.getName()`
+   value. The rest of the path MUST NOT be validated.
+1. Use any status fields as desired -for example the file length.
+
+Even if not values of the status are used, the presence of the argument
+can be interpreted as the caller declaring that they believe the file
+to be present and of the given size.
 
 ## Builder interface
 
@@ -128,26 +191,499 @@ completed, returns an input stream which can read data from the filesystem.
 
 The `build()` operation MAY perform the validation of the file's existence,
 its kind, so rejecting attempts to read from a directory or non-existent
-file. **Alternatively**, the `build()` operation may delay all checks
-until an asynchronous operation whose outcome is provided by the `Future`
+file. Alternatively
+* file existence/status checks MAY be performed asynchronously within the returned
+    `CompletableFuture<>`.
+* file existence/status checks MAY be postponed until the first byte is read in
+  any of the read such as `read()` or `PositionedRead`.
 
 That is, the precondition  `exists(FS, path)` and `isFile(FS, path)` are
-only guaranteed to have been met after the `get()` on the returned future is successful.
+only guaranteed to have been met after the `get()` called on returned future
+and an attempt has been made to read the stream.
 
-Thus, if even a file does not exist, the following call will still succeed, returning
-a future to be evaluated.
+Thus, if even when file does not exist, or is a directory rather than a file,
+the following call MUST succeed, returning a `CompletableFuture` to be evaluated.
 
 ```java
 Path p = new Path("file://tmp/file-which-does-not-exist");
 
 CompletableFuture<FSDataInputStream> future = p.getFileSystem(conf)
       .openFile(p)
-      .build;
+      .build();
 ```
 
-The preconditions for opening the file are checked during the asynchronous
-evaluation, and so will surface when the future is completed:
+The inability to access/read a file MUST raise an `IOException`or subclass
+in either the future's `get()` call, or, for late binding operations,
+when an operation to read data is invoked.
+
+Therefore the following sequence SHALL fail when invoked on the
+`future` returned by the previous example.
 
 ```java
-FSDataInputStream in = future.get();
+  future.get().read();
 ```
+
+Access permission checks have the same visibility requirements: permission failures
+MUST be delayed until the `get()` call and MAY be delayed into subsequent operations.
+
+Note: some operations on the input stream, such as `seek()` may not attempt any IO
+at all. Such operations MAY NOT raise exceotions when interacting with
+nonexistent/unreadable files.
+
+## <a name="options"></a> Standard `openFile()` options since Hadoop 3.3.3
+
+These are options which `FileSystem` and `FileContext` implementation
+MUST recognise and MAY support by changing the behavior of
+their input streams as appropriate.
+
+Hadoop 3.3.0 added the `openFile()` API; these standard options were defined in
+a later release. Therefore, although they are "well known", unless confident that
+the application will only be executed against releases of Hadoop which knows of
+the options -applications SHOULD set the options via `opt()` calls rather than `must()`.
+
+When opening a file through the `openFile()` builder API, callers MAY use
+both `.opt(key, value)` and `.must(key, value)` calls to set standard and
+filesystem-specific options.
+
+If set as an `opt()` parameter, unsupported "standard" options MUST be ignored,
+as MUST unrecognized standard options.
+
+If set as a `must()` parameter, unsupported "standard" options MUST be ignored.
+unrecognized standard options MUST be rejected.
+
+The standard `openFile()` options are defined
+in `org.apache.hadoop.fs.OpenFileOptions`; they all SHALL start
+with `fs.option.openfile.`.
+
+Note that while all `FileSystem`/`FileContext` instances SHALL support these
+options to the extent that `must()` declarations SHALL NOT fail, the
+implementations MAY support them to the extent of interpreting the values. This
+means that it is not a requirement for the stores to actually read the read
+policy or file length values and use them when opening files.
+
+Unless otherwise stated, they SHOULD be viewed as hints.
+
+Note: if a standard option is added such that if set but not
+supported would be an error, then implementations SHALL reject it. For example,
+the S3A filesystem client supports the ability to push down SQL commands. If
+something like that were ever standardized, then the use of the option, either
+in `opt()` or `must()` argument MUST be rejected for filesystems which don't
+support the feature.
+
+### <a name="buffer.size"></a>  Option: `fs.option.openfile.buffer.size`
+
+Read buffer size in bytes.
+
+This overrides the default value set in the configuration with the option
+`io.file.buffer.size`.
+
+It is supported by all filesystem clients which allow for stream-specific buffer
+sizes to be set via `FileSystem.open(path, buffersize)`.
+
+### <a name="read.policy"></a> Option: `fs.option.openfile.read.policy`
+
+Declare the read policy of the input stream. This is a hint as to what the
+expected read pattern of an input stream will be. This MAY control readahead,
+buffering and other optimizations.
+
+Sequential reads may be optimized with prefetching data and/or reading data in
+larger blocks. Some applications (e.g. distCp) perform sequential IO even over
+columnar data.
+
+In contrast, random IO reads data in different parts of the file using a
+sequence of `seek()/read()`
+or via the `PositionedReadable` or `ByteBufferPositionedReadable` APIs.
+
+Random IO performance may be best if little/no prefetching takes place, along
+with other possible optimizations
+
+Queries over columnar formats such as Apache ORC and Apache Parquet perform such
+random IO; other data formats may be best read with sequential or whole-file
+policies.
+
+What is key is that optimizing reads for seqential reads may impair random
+performance -and vice versa.
+
+1. The seek policy is a hint; even if declared as a `must()` option, the
+   filesystem MAY ignore it.
+1. The interpretation/implementation of a policy is a filesystem specific
+   behavior -and it may change with Hadoop releases and/or specific storage
+   subsystems.
+1. If a policy is not recognized, the filesystem client MUST ignore it.
+
+| Policy       | Meaning                                                  |
+|--------------|----------------------------------------------------------|
+| `adaptive`   | Any adaptive policy implemented by the store.            |
+| `default`    | The default policy for this store. Generally "adaptive". |
+| `random`     | Optimize for random access.                              |
+| `sequential` | Optimize for sequential access.                          |
+| `vector`     | The Vectored IO API is intended to be used.              |
+| `whole-file` | The whole file will be read.                             |
+
+Choosing the wrong read policy for an input source may be inefficient.
+
+A list of read policies MAY be supplied; the first one recognized/supported by
+the filesystem SHALL be the one used. This allows for custom policies to be
+supported, for example an `hbase-hfile` policy optimized for HBase HFiles.
+
+The S3A and ABFS input streams both implement
+the [IOStatisticsSource](iostatistics.html) API, and can be queried for their IO
+Performance.
+
+*Tip:* log the `toString()` value of input streams at `DEBUG`. The S3A and ABFS
+Input Streams log read statistics, which can provide insight about whether reads
+are being performed efficiently or not.
+
+_Futher reading_
+
+* [Linux fadvise()](https://linux.die.net/man/2/fadvise).
+* [Windows `CreateFile()`](https://docs.microsoft.com/en-us/windows/win32/api/fileapi/nf-fileapi-createfilea#caching-behavior)
+
+#### <a name="read.policy."></a> Read Policy `adaptive`
+
+Try to adapt the seek policy to the read pattern of the application.
+
+The `normal` policy of the S3A client and the sole policy supported by
+the `wasb:` client are both adaptive -they assume sequential IO, but once a
+backwards seek/positioned read call is made the stream switches to random IO.
+
+Other filesystem implementations may wish to adopt similar strategies, and/or
+extend the algorithms to detect forward seeks and/or switch from random to
+sequential IO if that is considered more efficient.
+
+Adaptive read policies are the absence of the ability to
+declare the seek policy in the `open()` API, so requiring it to be declared, if
+configurable, in the cluster/application configuration. However, the switch from
+sequential to random seek policies may be exensive.
+
+When applications explicitly set the `fs.option.openfile.read.policy` option, if
+they know their read plan, they SHOULD declare which policy is most appropriate.
+
+#### <a name="read.policy.default"></a> Read Policy ``
+
+The default policy for the filesystem instance.
+Implementation/installation-specific.
+
+#### <a name="read.policy.sequential"></a> Read Policy `sequential`
+
+Expect sequential reads from the first byte read to the end of the file/until
+the stream is closed.
+
+#### <a name="read.policy.random"></a> Read Policy `random`
+
+Expect `seek()/read()` sequences, or use of `PositionedReadable`
+or `ByteBufferPositionedReadable` APIs.
+
+
+#### <a name="read.policy.vector"></a> Read Policy `vector`
+
+This declares that the caller intends to use the Vectored read API of
+[HADOOP-11867](https://issues.apache.org/jira/browse/HADOOP-11867)
+_Add a high-performance vectored read API_.
+
+This is a hint: it is not a requirement when using the API.
+It does inform the implemenations that the stream should be
+configured for optimal vectored IO performance, if such a
+feature has been implemented.
+
+It is *not* exclusive: the same stream may still be used for
+classic `InputStream` and `PositionedRead` API calls.
+Implementations SHOULD use the `random` read policy
+with these operations.
+
+#### <a name="read.policy.whole-file"></a> Read Policy `whole-file`
+
+
+This declares that the whole file is to be read end-to-end; the file system client is free to enable
+whatever strategies maximise performance for this. In particular, larger ranged reads/GETs can
+deliver high bandwidth by reducing socket/TLS setup costs and providing a connection long-lived
+enough for TCP flow control to determine the optimal download rate.
+
+Strategies can include:
+
+* Initiate an HTTP GET of the entire file in `openFile()` operation.
+* Prefech data in large blocks, possibly in parallel read operations.
+
+Applications which know that the entire file is to be read from an opened stream SHOULD declare this
+read policy.
+
+### <a name="openfile.length"></a> Option: `fs.option.openfile.length`
+
+Declare the length of a file.
+
+This can be used by clients to skip querying a remote store for the size
+of/existence of a file when opening it, similar to declaring a file status
+through the `withFileStatus()` option.
+
+If supported by a filesystem connector, this option MUST be interpreted as
+declaring the minimum length of the file:
+
+1. If the value is negative, the option SHALL be considered unset.
+2. It SHALL NOT be an error if the actual length of the file is greater than
+   this value.
+3. `read()`, `seek()` and positioned read calls MAY use a position across/beyond
+   this length but below the actual length of the file. Implementations MAY
+   raise `EOFExceptions` in such cases, or they MAY return data.
+
+If this option is used by the FileSystem implementation
+
+*Implementor's Notes*
+
+* A value of `fs.option.openfile.length` &lt; 0 MUST be rejected.
+* If a file status is supplied along with a value in `fs.opt.openfile.length`;
+  the file status values take precedence.
+
+### <a name="split.start"></a> Options: `fs.option.openfile.split.start` and `fs.option.openfile.split.end`
+
+Declare the start and end of the split when a file has been split for processing
+in pieces.
+
+1. If a value is negative, the option SHALL be considered unset.
+1. Filesystems MAY assume that the length of the file is greater than or equal
+   to the value of `fs.option.openfile.split.end`.
+1. And that they MAY raise an exception if the client application reads past the
+   value set in `fs.option.openfile.split.end`.
+1. The pair of options MAY be used to optimise the read plan, such as setting
+   the content range for GET requests, or using the split end as an implicit
+   declaration of the guaranteed minimum length of the file.
+1. If both options are set, and the split start is declared as greater than the
+   split end, then the split start SHOULD just be reset to zero, rather than
+   rejecting the operation.
+
+The split end value can provide a hint as to the end of the input stream. The
+split start can be used to optimize any initial read offset for filesystem
+clients.
+
+*Note for implementors: applications will read past the end of a split when they
+need to read to the end of a record/line which begins before the end of the
+split.
+
+Therefore clients MUST be allowed to `seek()`/`read()` past the length
+set in `fs.option.openfile.split.end` if the file is actually longer
+than that value.
+
+## <a name="s3a"></a> S3A-specific options
+
+The S3A Connector supports custom options for readahead and seek policy.
+
+| Name                                 | Type     | Meaning                                                     |
+|--------------------------------------|----------|-------------------------------------------------------------|
+| `fs.s3a.readahead.range`             | `long`   | readahead range in bytes                                    |
+| `fs.s3a.input.async.drain.threshold` | `long`   | threshold to switch to asynchronous draining of the stream  |
+| `fs.s3a.experimental.input.fadvise`  | `String` | seek policy. Superceded by `fs.option.openfile.read.policy` |
+
+If the option set contains a SQL statement in the `fs.s3a.select.sql` statement,
+then the file is opened as an S3 Select query.
+Consult the S3A documentation for more details.
+
+## <a name="abfs"></a> ABFS-specific options
+
+The ABFS Connector supports custom input stream options.
+
+| Name                              | Type      | Meaning                                            |
+|-----------------------------------|-----------|----------------------------------------------------|
+| `fs.azure.buffered.pread.disable` | `boolean` | disable caching on the positioned read operations. |
+
+
+Disables caching on data read through the [PositionedReadable](fsdatainputstream.html#PositionedReadable)
+APIs.
+
+Consult the ABFS Documentation for more details.
+
+## <a name="examples"></a> Examples
+
+#### Declaring seek policy and split limits when opening a file.
+
+Here is an example from a proof of
+concept `org.apache.parquet.hadoop.util.HadoopInputFile`
+reader which uses a (nullable) file status and a split start/end.
+
+The `FileStatus` value is always passed in -but if it is null, then the split
+end is used to declare the length of the file.
+
+```java
+protected SeekableInputStream newStream(Path path, FileStatus stat,
+     long splitStart, long splitEnd)
+     throws IOException {
+
+   FutureDataInputStreamBuilder builder = fs.openFile(path)
+   .opt("fs.option.openfile.read.policy", "vector, random")
+   .withFileStatus(stat);
+
+   builder.opt("fs.option.openfile.split.start", splitStart);
+   builder.opt("fs.option.openfile.split.end", splitEnd);
+   CompletableFuture<FSDataInputStream> streamF = builder.build();
+   return HadoopStreams.wrap(FutureIO.awaitFuture(streamF));
+}
+```
+
+As a result, whether driven directly by a file listing, or when opening a file
+from a query plan of `(path, splitStart, splitEnd)`, there is no need to probe
+the remote store for the length of the file. When working with remote object
+stores, this can save tens to hundreds of milliseconds, even if such a probe is
+done asynchronously.
+
+If both the file length and the split end is set, then the file length MUST be
+considered "more" authoritative, that is it really SHOULD be defining the file
+length. If the split end is set, the caller MAY ot read past it.
+
+The `CompressedSplitLineReader` can read past the end of a split if it is
+partway through processing a compressed record. That is: it assumes an
+incomplete record read means that the file length is greater than the split
+length, and that it MUST read the entirety of the partially read record. Other
+readers may behave similarly.
+
+Therefore
+
+1. File length as supplied in a `FileStatus` or in `fs.option.openfile.length`
+   SHALL set the strict upper limit on the length of a file
+2. The split end as set in `fs.option.openfile.split.end` MUST be viewed as a
+   hint, rather than the strict end of the file.
+
+### Opening a file with both standard and non-standard options
+
+Standard and non-standard options MAY be combined in the same `openFile()`
+operation.
+
+```java
+Future<FSDataInputStream> f = openFile(path)
+  .must("fs.option.openfile.read.policy", "random, adaptive")
+  .opt("fs.s3a.readahead.range", 1024 * 1024)
+  .build();
+
+FSDataInputStream is = f.get();
+```
+
+The option set in `must()` MUST be understood, or at least recognized and
+ignored by all filesystems. In this example, S3A-specific option MAY be
+ignored by all other filesystem clients.
+
+### Opening a file with older releases
+
+Not all hadoop releases recognize the `fs.option.openfile.read.policy` option.
+
+The option can be safely used in application code if it is added via the `opt()`
+builder argument, as it will be treated as an unknown optional key which can
+then be discarded.
+
+```java
+Future<FSDataInputStream> f = openFile(path)
+  .opt("fs.option.openfile.read.policy", "vector, random, adaptive")
+  .build();
+
+FSDataInputStream is = f.get();
+```
+
+*Note 1* if the option name is set by a reference to a constant in
+`org.apache.hadoop.fs.Options.OpenFileOptions`, then the program will not link
+against versions of Hadoop without the specific option. Therefore for resilient
+linking against older releases -use a copy of the value.
+
+*Note 2* as option validation is performed in the FileSystem connector,
+a third-party connector designed to work with multiple hadoop versions
+MAY NOT support the option.
+
+### Passing options in to MapReduce
+
+Hadoop MapReduce will automatically read MR Job Options with the prefixes
+`mapreduce.job.input.file.option.` and `mapreduce.job.input.file.must.`
+prefixes, and apply these values as `.opt()` and `must()` respectively, after
+remove the mapreduce-specific prefixes.
+
+This makes passing options in to MR jobs straightforward. For example, to
+declare that a job should read its data using random IO:
+
+```java
+JobConf jobConf = (JobConf) job.getConfiguration()
+jobConf.set(
+    "mapreduce.job.input.file.option.fs.option.openfile.read.policy",
+    "random");
+```
+
+### MapReduce input format propagating options
+
+An example of a record reader passing in options to the file it opens.
+
+```java
+  public void initialize(InputSplit genericSplit,
+                     TaskAttemptContext context) throws IOException {
+    FileSplit split = (FileSplit)genericSplit;
+    Configuration job = context.getConfiguration();
+    start = split.getStart();
+    end = start + split.getLength();
+    Path file = split.getPath();
+
+    // open the file and seek to the start of the split
+    FutureDataInputStreamBuilder builder =
+      file.getFileSystem(job).openFile(file);
+    // the start and end of the split may be used to build
+    // an input strategy.
+    builder.opt("fs.option.openfile.split.start", start);
+    builder.opt("fs.option.openfile.split.end", end);
+    FutureIO.propagateOptions(builder, job,
+        "mapreduce.job.input.file.option",
+        "mapreduce.job.input.file.must");
+
+    fileIn = FutureIO.awaitFuture(builder.build());
+    fileIn.seek(start)
+    /* Rest of the operation on the opened stream */
+  }
+```
+
+### `FileContext.openFile`
+
+From `org.apache.hadoop.fs.AvroFSInput`; a file is opened with sequential input.
+Because the file length has already been probed for, the length is passd down
+
+```java
+  public AvroFSInput(FileContext fc, Path p) throws IOException {
+    FileStatus status = fc.getFileStatus(p);
+    this.len = status.getLen();
+    this.stream = awaitFuture(fc.openFile(p)
+        .opt("fs.option.openfile.read.policy",
+            "sequential")
+        .opt("fs.option.openfile.length",
+            Long.toString(status.getLen()))
+        .build());
+    fc.open(p);
+  }
+```
+
+In this example, the length is passed down as a string (via `Long.toString()`)
+rather than directly as a long. This is to ensure that the input format will
+link against versions of $Hadoop which do not have the
+`opt(String, long)` and `must(String, long)` builder parameters. Similarly, the
+values are passed as optional, so that if unrecognized the application will
+still succeed.
+
+### Example: reading a whole file
+
+This is from `org.apache.hadoop.util.JsonSerialization`.
+
+Its `load(FileSystem, Path, FileStatus)` method
+* declares the whole file is to be read end to end.
+* passes down the file status
+
+```java
+public T load(FileSystem fs,
+        Path path,
+        status)
+        throws IOException {
+
+ try (FSDataInputStream dataInputStream =
+          awaitFuture(fs.openFile(path)
+              .opt("fs.option.openfile.read.policy", "whole-file")
+              .withFileStatus(status)
+              .build())) {
+   return fromJsonStream(dataInputStream);
+ } catch (JsonProcessingException e) {
+   throw new PathIOException(path.toString(),
+       "Failed to read JSON file " + e, e);
+ }
+}
+```
+
+*Note:* : in Hadoop 3.3.2 and earlier, the `withFileStatus(status)` call
+required a non-null parameter; this has since been relaxed.
+For maximum compatibility across versions, only invoke the method
+when the file status is known to be non-null.
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
index a4aa136033a..e18f4c3bf4a 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
@@ -41,3 +41,4 @@ HDFS as these are commonly expected by Hadoop client applications.
 2. [Extending the specification and its tests](extending.html)
 1. [Uploading a file using Multiple Parts](multipartuploader.html)
 1. [IOStatistics](iostatistics.html)
+1. [openFile()](openfile.html).
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
new file mode 100644
index 00000000000..afb3245c510
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
@@ -0,0 +1,122 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# `FileSystem.openFile()`/`FileContext.openFile()`
+
+This is a method provided by both FileSystem and FileContext for
+advanced file opening options and, where implemented,
+an asynchrounous/lazy opening of a file.
+
+Creates a builder to open a file, supporting options
+both standard and filesystem specific. The return
+value of the `build()` call is a `Future<FSDataInputStream>`,
+which must be waited on. The file opening may be
+asynchronous, and it may actually be postponed (including
+permission/existence checks) until reads are actually
+performed.
+
+This API call was added to `FileSystem` and `FileContext` in
+Hadoop 3.3.0; it was tuned in Hadoop 3.3.1 as follows.
+
+* Added `opt(key, long)` and `must(key, long)`.
+* Declared that `withFileStatus(null)` is allowed.
+* Declared that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+* Added standard option keys.
+
+###  <a name="openfile_path_"></a> `FutureDataInputStreamBuilder openFile(Path path)`
+
+Creates a [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FutureDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`FileSystem.openFileWithOptions(Path, OpenFileParameters)` or
+`AbstractFileSystem.openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+These protected methods returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `FileSystem.openFileWithOptions(PathHandle, OpenFileParameters)`
+ultimately invokes `FileSystem.open(Path, int)`.
+
+Thus the chain `FileSystem.openFile(path).build().get()` has the same preconditions
+and postconditions as `FileSystem.open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of:
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+This saves network IO on object stores.
+
+The `openFile()` operation MAY check the state of the filesystem during its
+invocation, but as the state of the filesystem may change between this call and
+the actual `build()` and `get()` operations, this file-specific
+preconditions (file exists, file is readable, etc) MUST NOT be checked here.
+
+FileSystem implementations which do not implement `open(Path, int)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FutureDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+Consult [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html) for details
+on how to use the builder, and for standard options which may be passed in.
+
+### <a name="openfile_pathhandle_"></a> `FutureDataInputStreamBuilder openFile(PathHandle)`
+
+Creates a [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file identified by the given `PathHandle` for reading.
+
+If implemented by a filesystem, the semantics of  [`openFile(Path)`](#openfile_path_)
+Thus the chain `openFile(pathhandle).build().get()` has the same preconditions and postconditions
+as `open(Pathhandle, int)`
+
+FileSystem implementations which do not implement `open(PathHandle handle, int bufferSize)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FutureDataInputStreamBuilder.build()` or the subsequent `get()` call, else they MAY fail fast in
+the `openFile(PathHandle)` call.
+
+The base implementation raises this exception in the `build()` operation; other implementations
+SHOULD copy this.
+
+### Implementors notes
+
+The base implementation of `openFileWithOptions()` actually executes
+the `open(path)` operation synchronously, yet still returns the result
+or any failures in the `CompletableFuture<>`, so as to provide a consistent
+lifecycle across all filesystems.
+
+Any filesystem client where the time to open a file may be significant SHOULD
+execute it asynchronously by submitting the operation in some executor/thread
+pool. This is particularly recommended for object stores and other filesystems
+likely to be accessed over long-haul connections.
+
+Arbitrary filesystem-specific options MAY be supported; these MUST
+be prefixed with either the filesystem schema, e.g. `hdfs.`
+or in the `fs.SCHEMA` format as normal configuration settings `fs.hdfs`. The
+latter style allows the same configuration option to be used for both
+filesystem configuration and file-specific configuration.
+
+It SHOULD be possible to always open a file without specifying any options,
+so as to present a consistent model to users. However, an implementation MAY
+opt to require one or more mandatory options to be set.
+
+The returned stream may perform "lazy" evaluation of file access. This is
+relevant for object stores where the probes for existence are expensive, and,
+even with an asynchronous open, may be considered needless.
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java
index 3e754e4578d..c395afdb377 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java
@@ -50,11 +50,11 @@ import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.util.DurationInfo;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists;
-import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture;
 import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
 import static org.apache.hadoop.test.LambdaTestUtils.eventually;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /**
  * Tests of multipart uploads.
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
index a43053180fb..25bfe082b01 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
@@ -30,14 +30,18 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.impl.FutureIOSupport;
 import org.apache.hadoop.io.IOUtils;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.compareByteArrays;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 import org.junit.Test;
 
@@ -232,7 +236,7 @@ public abstract class AbstractContractOpenTest
         getFileSystem().openFile(path("testAwaitFutureFailToFNFE"))
             .opt("fs.test.something", true);
     intercept(FileNotFoundException.class,
-        () -> FutureIOSupport.awaitFuture(builder.build()));
+        () -> awaitFuture(builder.build()));
   }
 
   @Test
@@ -242,7 +246,7 @@ public abstract class AbstractContractOpenTest
         getFileSystem().openFile(path("testAwaitFutureFailToFNFE"))
             .opt("fs.test.something", true);
     intercept(FileNotFoundException.class,
-        () -> FutureIOSupport.awaitFuture(builder.build(),
+        () -> awaitFuture(builder.build(),
             10, TimeUnit.DAYS));
   }
 
@@ -250,7 +254,7 @@ public abstract class AbstractContractOpenTest
   public void testOpenFileExceptionallyTranslating() throws Throwable {
     describe("openFile missing file chains into exceptionally()");
     CompletableFuture<FSDataInputStream> f = getFileSystem()
-        .openFile(path("testOpenFileUnknownOption")).build();
+        .openFile(path("testOpenFileExceptionallyTranslating")).build();
     interceptFuture(RuntimeException.class,
         "exceptionally",
         f.exceptionally(ex -> {
@@ -262,11 +266,12 @@ public abstract class AbstractContractOpenTest
   public void testChainedFailureAwaitFuture() throws Throwable {
     describe("await Future handles chained failures");
     CompletableFuture<FSDataInputStream> f = getFileSystem()
-        .openFile(path("testOpenFileUnknownOption"))
+        .openFile(path("testChainedFailureAwaitFuture"))
+        .withFileStatus(null)
         .build();
     intercept(RuntimeException.class,
         "exceptionally",
-        () -> FutureIOSupport.awaitFuture(
+        () -> awaitFuture(
             f.exceptionally(ex -> {
               throw new RuntimeException("exceptionally", ex);
             })));
@@ -280,13 +285,34 @@ public abstract class AbstractContractOpenTest
     int len = 4096;
     createFile(fs, path, true,
         dataset(len, 0x40, 0x80));
+    FileStatus st = fs.getFileStatus(path);
     CompletableFuture<Long> readAllBytes = fs.openFile(path)
-        .withFileStatus(fs.getFileStatus(path))
+        .withFileStatus(st)
         .build()
         .thenApply(ContractTestUtils::readStream);
     assertEquals("Wrong number of bytes read value",
         len,
         (long) readAllBytes.get());
+    // now reattempt with a new FileStatus and a different path
+    // other than the final name element
+    // implementations MUST use path in openFile() call
+    FileStatus st2 = new FileStatus(
+        len, false,
+        st.getReplication(),
+        st.getBlockSize(),
+        st.getModificationTime(),
+        st.getAccessTime(),
+        st.getPermission(),
+        st.getOwner(),
+        st.getGroup(),
+        new Path("gopher:///localhost:/" + path.getName()));
+    assertEquals("Wrong number of bytes read value",
+        len,
+        (long) fs.openFile(path)
+            .withFileStatus(st2)
+            .build()
+            .thenApply(ContractTestUtils::readStream)
+            .get());
   }
 
   @Test
@@ -298,17 +324,47 @@ public abstract class AbstractContractOpenTest
         dataset(4, 0x40, 0x80));
     CompletableFuture<FSDataInputStream> future = fs.openFile(path).build();
     AtomicBoolean accepted = new AtomicBoolean(false);
-    future.thenAcceptAsync(i -> accepted.set(true)).get();
+    future.thenApply(stream -> {
+      accepted.set(true);
+      return stream;
+    }).get().close();
     assertTrue("async accept operation not invoked",
         accepted.get());
   }
 
+  /**
+   * Open a file with a null status, and the length
+   * passed in as an opt() option (along with sequential IO).
+   * The file is opened, the data read, and it must match
+   * the source data.
+   * opt() is used so that integration testing with external
+   * filesystem connectors will downgrade if the option is not
+   * recognized.
+   */
   @Test
-  public void testOpenFileNullStatus() throws Throwable {
-    describe("use openFile() with a null status");
+  public void testOpenFileNullStatusButFileLength() throws Throwable {
+    describe("use openFile() with a null status and expect the status to be"
+        + " ignored. block size, fadvise and length are passed in as"
+        + " opt() options");
     Path path = path("testOpenFileNullStatus");
-    intercept(NullPointerException.class,
-        () -> getFileSystem().openFile(path).withFileStatus(null));
+    FileSystem fs = getFileSystem();
+    int len = 4;
+    byte[] result = new byte[len];
+    byte[] dataset = dataset(len, 0x40, 0x80);
+    createFile(fs, path, true,
+        dataset);
+    CompletableFuture<FSDataInputStream> future = fs.openFile(path)
+        .withFileStatus(null)
+        .opt(FS_OPTION_OPENFILE_READ_POLICY,
+            "unknown, sequential, random")
+        .opt(FS_OPTION_OPENFILE_BUFFER_SIZE, 32768)
+        .opt(FS_OPTION_OPENFILE_LENGTH, len)
+        .build();
+
+    try (FSDataInputStream in = future.get()) {
+      in.readFully(result);
+    }
+    compareByteArrays(dataset, result, len);
   }
 
 }
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 e13a49ca10e..eb56d957d9a 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
@@ -1642,17 +1642,22 @@ public class ContractTestUtils extends Assert {
 
   /**
    * Read a whole stream; downgrades an IOE to a runtime exception.
+   * Closes the stream afterwards.
    * @param in input
    * @return the number of bytes read.
    * @throws AssertionError on any IOException
    */
   public static long readStream(InputStream in) {
-    long count = 0;
+    try {
+      long count = 0;
 
-    while (read(in) >= 0) {
-      count++;
+      while (read(in) >= 0) {
+        count++;
+      }
+      return count;
+    } finally {
+      IOUtils.cleanupWithLogger(LOG, in);
     }
-    return count;
   }
 
 
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java
index 22f6c33d2e2..755599f0c39 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java
@@ -36,6 +36,8 @@ import org.assertj.core.api.ObjectAssert;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /**
@@ -347,6 +349,24 @@ public final class IOStatisticAssertions {
         verifyStatisticsNotNull(stats).maximums());
   }
 
+  /**
+   * Assert that a duration is within a given minimum/maximum range.
+   * @param stats statistics source
+   * @param key statistic key without any suffix
+   * @param min minimum statistic must be equal to or greater than this.
+   * @param max maximum statistic must be equal to or less than this.
+   */
+  public static void assertDurationRange(
+      final IOStatistics stats,
+      final String key,
+      final long min,
+      final long max) {
+    assertThatStatisticMinimum(stats, key + SUFFIX_MIN)
+        .isGreaterThanOrEqualTo(min);
+    assertThatStatisticMaximum(stats, key + SUFFIX_MAX)
+        .isLessThanOrEqualTo(max);
+  }
+
   /**
    * Start an assertion chain on
    * a required mean statistic.
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java
index 8258b62c1f7..cfde1583e2c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java
@@ -30,7 +30,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.fs.impl.FutureIOSupport;
 import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
 import org.apache.hadoop.test.AbstractHadoopTestBase;
 import org.apache.hadoop.util.functional.FunctionRaisingIOE;
@@ -276,7 +275,7 @@ public class TestDurationTracking extends AbstractHadoopTestBase {
    */
   @Test
   public void testDurationThroughEval() throws Throwable {
-    CompletableFuture<Object> eval = FutureIOSupport.eval(
+    CompletableFuture<Object> eval = FutureIO.eval(
         trackDurationOfOperation(stats, REQUESTS, () -> {
           sleepf(100);
           throw new FileNotFoundException("oops");


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


[hadoop] 03/04: HADOOP-16202. Enhanced openFile(): hadoop-aws changes. (#2584/3)

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

stevel pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit e0cd0a82e032b926774dcea69edd4fa20aae2603
Author: Steve Loughran <st...@cloudera.com>
AuthorDate: Sun Apr 24 17:23:19 2022 +0100

    HADOOP-16202. Enhanced openFile(): hadoop-aws changes. (#2584/3)
    
    S3A input stream support for the few fs.option.openfile settings.
    As well as supporting the read policy option and values,
    if the file length is declared in fs.option.openfile.length
    then no HEAD request will be issued when opening a file.
    This can cut a few tens of milliseconds off the operation.
    
    The patch adds a new openfile parameter/FS configuration option
    fs.s3a.input.async.drain.threshold (default: 16000).
    It declares the number of bytes remaining in the http input stream
    above which any operation to read and discard the rest of the stream,
    "draining", is executed asynchronously.
    This asynchronous draining offers some performance benefit on seek-heavy
    file IO.
    
    Contributed by Steve Loughran.
    
    Change-Id: I9b0626bbe635e9fd97ac0f463f5e7167e0111e39
---
 .../hadoop-aws/dev-support/findbugs-exclude.xml    |   5 +
 .../java/org/apache/hadoop/fs/s3a/Constants.java   |  45 +-
 .../java/org/apache/hadoop/fs/s3a/Invoker.java     |  28 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java    | 283 +++++-----
 .../org/apache/hadoop/fs/s3a/S3AInputPolicy.java   |  93 +++-
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   | 263 ++++++---
 .../apache/hadoop/fs/s3a/S3AInstrumentation.java   |  12 +-
 .../org/apache/hadoop/fs/s3a/S3AReadOpContext.java | 101 +++-
 .../apache/hadoop/fs/s3a/S3ObjectAttributes.java   |  15 +-
 .../java/org/apache/hadoop/fs/s3a/Statistic.java   |  17 +
 .../hadoop/fs/s3a/commit/CommitOperations.java     |   8 +-
 .../hadoop/fs/s3a/commit/files/PendingSet.java     |  26 +-
 .../fs/s3a/commit/files/SinglePendingCommit.java   |  20 +-
 .../hadoop/fs/s3a/impl/AbstractStoreOperation.java |  26 +-
 .../hadoop/fs/s3a/impl/CallableSupplier.java       |   2 +-
 .../hadoop/fs/s3a/impl/InternalConstants.java      |  21 +-
 .../apache/hadoop/fs/s3a/impl/OpenFileSupport.java | 600 +++++++++++++++++++++
 .../apache/hadoop/fs/s3a/s3guard/S3GuardTool.java  |   4 +-
 .../fs/s3a/select/InternalSelectConstants.java     |   2 +-
 .../apache/hadoop/fs/s3a/select/SelectTool.java    |   4 +-
 .../s3a/statistics/S3AInputStreamStatistics.java   |   7 +
 .../statistics/impl/EmptyS3AStatisticsContext.java |   4 +
 .../src/site/markdown/tools/hadoop-aws/index.md    |   8 +
 .../fs/contract/s3a/ITestS3AContractOpen.java      |  67 +++
 .../fs/contract/s3a/ITestS3AContractSeek.java      |  15 +-
 .../apache/hadoop/fs/s3a/AbstractS3AMockTest.java  |   4 +
 .../hadoop/fs/s3a/ITestS3AConfiguration.java       |  11 -
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java     |   4 +-
 .../hadoop/fs/s3a/TestS3AInputStreamRetry.java     |  17 +-
 .../org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java  |   5 +-
 .../hadoop/fs/s3a/TestStreamChangeTracker.java     |   2 +-
 .../hadoop/fs/s3a/impl/TestOpenFileSupport.java    | 429 +++++++++++++++
 .../fs/s3a/performance/ITestS3AOpenCost.java       | 209 +++++++
 .../hadoop/fs/s3a/performance/OperationCost.java   |   6 +
 .../s3a/scale/ITestS3AInputStreamPerformance.java  |  58 +-
 .../hadoop/fs/s3a/select/AbstractS3SelectTest.java |   2 +-
 .../apache/hadoop/fs/s3a/select/ITestS3Select.java |   7 +-
 .../hadoop/fs/s3a/select/ITestS3SelectMRJob.java   |   4 +-
 38 files changed, 2062 insertions(+), 372 deletions(-)

diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
index b4568b69de4..324369076b8 100644
--- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
@@ -28,6 +28,11 @@
     <Method name="s3Exists" />
     <Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE" />
   </Match>
+  <!-- we are using completable futures, so ignore the Future which submit() returns -->
+  <Match>
+    <Class name="org.apache.hadoop.fs.s3a.S3AFileSystem$InputStreamCallbacksImpl" />
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
+  </Match>
 
   <!--
    findbugs gets confused by lambda expressions in synchronized methods
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 cb3d72e5668..e5369b84883 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
 
 import java.util.concurrent.TimeUnit;
@@ -602,37 +603,69 @@ public final class Constants {
   public static final String READAHEAD_RANGE = "fs.s3a.readahead.range";
   public static final long DEFAULT_READAHEAD_RANGE = 64 * 1024;
 
+  /**
+   * The threshold at which drain operations switch
+   * to being asynchronous with the schedule/wait overhead
+   * compared to synchronous.
+   * Value: {@value}
+   */
+  public static final String ASYNC_DRAIN_THRESHOLD = "fs.s3a.input.async.drain.threshold";
+
+  /**
+   * This is a number based purely on experimentation in
+   * {@code ITestS3AInputStreamPerformance}.
+   * Value: {@value}
+   */
+  public static final int DEFAULT_ASYNC_DRAIN_THRESHOLD = 16_000;
+
   /**
    * Which input strategy to use for buffering, seeking and similar when
    * reading data.
    * Value: {@value}
    */
-  @InterfaceStability.Unstable
   public static final String INPUT_FADVISE =
       "fs.s3a.experimental.input.fadvise";
 
+  /**
+   * The default value for this FS.
+   * Which for S3A, is adaptive.
+   * Value: {@value}
+   * @deprecated use the {@link Options.OpenFileOptions} value
+   * in code which only needs to be compiled against newer hadoop
+   * releases.
+   */
+  public static final String INPUT_FADV_DEFAULT =
+      Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT;
+
   /**
    * General input. Some seeks, some reads.
+   * The policy name "default" is standard across different stores,
+   * and should be preferred.
    * Value: {@value}
    */
-  @InterfaceStability.Unstable
   public static final String INPUT_FADV_NORMAL = "normal";
 
   /**
    * Optimized for sequential access.
    * Value: {@value}
+   * @deprecated use the {@link Options.OpenFileOptions} value
+   * in code which only needs to be compiled against newer hadoop
+   * releases.
    */
-  @InterfaceStability.Unstable
-  public static final String INPUT_FADV_SEQUENTIAL = "sequential";
+  public static final String INPUT_FADV_SEQUENTIAL =
+      Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
 
   /**
    * Optimized purely for random seek+read/positionedRead operations;
    * The performance of sequential IO may be reduced in exchange for
    * more efficient {@code seek()} operations.
    * Value: {@value}
+   * @deprecated use the {@link Options.OpenFileOptions} value
+   * in code which only needs to be compiled against newer hadoop
+   * releases.
    */
-  @InterfaceStability.Unstable
-  public static final String INPUT_FADV_RANDOM = "random";
+  public static final String INPUT_FADV_RANDOM =
+      Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
 
   /**
    * Gauge name for the input policy : {@value}.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
index 0663fe935db..279bfeba987 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
@@ -31,6 +31,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.statistics.DurationTracker;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.util.DurationInfo;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
@@ -38,6 +39,8 @@ import org.apache.hadoop.util.functional.FutureIO;
 import org.apache.hadoop.util.functional.InvocationRaisingIOE;
 import org.apache.hadoop.util.Preconditions;
 
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
+
 /**
  * Class to provide lambda expression invocation of AWS operations.
  *
@@ -122,6 +125,31 @@ public class Invoker {
     }
   }
 
+  /**
+   * Execute a function, translating any exception into an IOException.
+   * The supplied duration tracker instance is updated with success/failure.
+   * @param action action to execute (used in error messages)
+   * @param path path of work (used in error messages)
+   * @param tracker tracker to update
+   * @param operation operation to execute
+   * @param <T> type of return value
+   * @return the result of the function call
+   * @throws IOException any IOE raised, or translated exception
+   */
+  @Retries.OnceTranslated
+  public static <T> T onceTrackingDuration(
+      final String action,
+      final String path,
+      final DurationTracker tracker,
+      final CallableRaisingIOE<T> operation)
+      throws IOException {
+    try {
+      return invokeTrackingDuration(tracker, operation);
+    } catch (AmazonClientException e) {
+      throw S3AUtils.translateException(action, path, e);
+    }
+  }
+
   /**
    * Execute an operation with no result.
    * @param action action to execute (used in error messages)
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 83c3a74f5b3..15e240f9018 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
@@ -90,6 +90,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Globber;
+import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
@@ -109,6 +110,7 @@ import org.apache.hadoop.fs.s3a.impl.HeaderProcessing;
 import org.apache.hadoop.fs.s3a.impl.InternalConstants;
 import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks;
 import org.apache.hadoop.fs.s3a.impl.MkdirOperation;
+import org.apache.hadoop.fs.s3a.impl.OpenFileSupport;
 import org.apache.hadoop.fs.s3a.impl.OperationCallbacks;
 import org.apache.hadoop.fs.s3a.impl.RenameOperation;
 import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl;
@@ -116,7 +118,6 @@ import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder;
 import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
 import org.apache.hadoop.fs.s3a.impl.StoreContext;
 import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
-import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
 import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations;
 import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
 import org.apache.hadoop.fs.statistics.DurationTracker;
@@ -169,6 +170,7 @@ import org.apache.hadoop.fs.s3a.select.SelectConstants;
 import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
 import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics;
+import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
 import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
 import org.apache.hadoop.fs.s3a.statistics.impl.BondedS3AStatisticsContext;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
@@ -187,7 +189,8 @@ import org.apache.hadoop.util.functional.CallableRaisingIOE;
 import static java.util.Objects.requireNonNull;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_DEFAULT;
-import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Invoker.*;
@@ -298,7 +301,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   /** Storage Statistics Bonded to the instrumentation. */
   private S3AStorageStatistics storageStatistics;
 
-  private long readAhead;
+  /**
+   * Default input policy; may be overridden in
+   * {@code openFile()}.
+   */
   private S3AInputPolicy inputPolicy;
   private ChangeDetectionPolicy changeDetectionPolicy;
   private final AtomicBoolean closed = new AtomicBoolean(false);
@@ -327,6 +333,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   private final ListingOperationCallbacks listingOperationCallbacks =
           new ListingOperationCallbacksImpl();
+
+  /**
+   * Helper for the openFile() method.
+   */
+  private OpenFileSupport openFileHelper;
+
   /**
    * Directory policy.
    */
@@ -465,9 +477,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       longBytesOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1);
       enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
 
-      readAhead = longBytesOption(conf, READAHEAD_RANGE,
-          DEFAULT_READAHEAD_RANGE, 0);
-
       initThreadPools(conf);
 
       int listVersion = conf.getInt(LIST_VERSION, DEFAULT_LIST_VERSION);
@@ -508,7 +517,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       doBucketProbing();
 
       inputPolicy = S3AInputPolicy.getPolicy(
-          conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL));
+          conf.getTrimmed(INPUT_FADVISE,
+              Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT),
+          S3AInputPolicy.Normal);
       LOG.debug("Input fadvise policy = {}", inputPolicy);
       changeDetectionPolicy = ChangeDetectionPolicy.getPolicy(conf);
       LOG.debug("Change detection policy = {}", changeDetectionPolicy);
@@ -555,6 +566,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       checkArgument(pageSize <= InternalConstants.MAX_ENTRIES_TO_DELETE,
               "page size out of range: %s", pageSize);
       listing = new Listing(listingOperationCallbacks, createStoreContext());
+      // now the open file logic
+      openFileHelper = new OpenFileSupport(
+          changeDetectionPolicy,
+          longBytesOption(conf, READAHEAD_RANGE,
+              DEFAULT_READAHEAD_RANGE, 0),
+          username,
+          intOption(conf, IO_FILE_BUFFER_SIZE_KEY,
+              IO_FILE_BUFFER_SIZE_DEFAULT, 0),
+          longBytesOption(conf, ASYNC_DRAIN_THRESHOLD,
+                        DEFAULT_ASYNC_DRAIN_THRESHOLD, 0),
+          inputPolicy);
     } catch (AmazonClientException e) {
       // amazon client exception: stop all services then throw the translation
       cleanupWithLogger(LOG, span);
@@ -1178,15 +1200,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     return fixBucketRegion(region);
   }
 
-  /**
-   * Returns the read ahead range value used by this filesystem.
-   * @return the readahead range
-   */
-  @VisibleForTesting
-  long getReadAheadRange() {
-    return readAhead;
-  }
-
   /**
    * Get the input policy for this FS instance.
    * @return the input policy
@@ -1268,13 +1281,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   /**
    * Change the input policy for this FS.
+   * This is now a no-op, retained in case some application
+   * or external test invokes it.
+   *
+   * @deprecated use openFile() options
    * @param inputPolicy new policy
    */
   @InterfaceStability.Unstable
+  @Deprecated
   public void setInputPolicy(S3AInputPolicy inputPolicy) {
-    Objects.requireNonNull(inputPolicy, "Null inputStrategy");
-    LOG.debug("Setting input strategy: {}", inputPolicy);
-    this.inputPolicy = inputPolicy;
+    LOG.warn("setInputPolicy is no longer supported");
   }
 
   /**
@@ -1392,64 +1408,46 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @Retries.RetryTranslated
   public FSDataInputStream open(Path f, int bufferSize)
       throws IOException {
-    return open(f, Optional.empty(), Optional.empty());
+    return executeOpen(qualify(f),
+        openFileHelper.openSimpleFile(bufferSize));
   }
 
   /**
    * Opens an FSDataInputStream at the indicated Path.
-   * if status contains an S3AFileStatus reference, it is used
-   * and so a HEAD request to the store is avoided.
-   *
-   * @param file the file to open
-   * @param options configuration options if opened with the builder API.
-   * @param providedStatus optional file status.
+   * The {@code fileInformation} parameter controls how the file
+   * is opened, whether it is normal vs. an S3 select call,
+   * can a HEAD be skipped, etc.
+   * @param path the file to open
+   * @param fileInformation information about the file to open
    * @throws IOException IO failure.
    */
-  @Retries.RetryTranslated
   @AuditEntryPoint
-  private FSDataInputStream open(
-      final Path file,
-      final Optional<Configuration> options,
-      final Optional<S3AFileStatus> providedStatus)
+  @Retries.RetryTranslated
+  private FSDataInputStream executeOpen(
+      final Path path,
+      final OpenFileSupport.OpenFileInformation fileInformation)
       throws IOException {
-
-    final Path path = qualify(file);
+    // create the input stream statistics before opening
+    // the file so that the time to prepare to open the file is included.
+    S3AInputStreamStatistics inputStreamStats =
+        statisticsContext.newInputStreamStatistics();
     // this span is passed into the stream.
     final AuditSpan auditSpan = entryPoint(INVOCATION_OPEN, path);
-    S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path,
-        providedStatus);
-
-    S3AReadOpContext readContext;
-    if (options.isPresent()) {
-      Configuration o = options.get();
-      // normal path. Open the file with the chosen seek policy, if different
-      // from the normal one.
-      // and readahead.
-      S3AInputPolicy policy = S3AInputPolicy.getPolicy(
-          o.get(INPUT_FADVISE, inputPolicy.toString()));
-      long readAheadRange2 = o.getLong(READAHEAD_RANGE, readAhead);
-      // TODO support change detection policy from options?
-      readContext = createReadContext(
-          fileStatus,
-          policy,
-          changeDetectionPolicy,
-          readAheadRange2,
-          auditSpan);
-    } else {
-      readContext = createReadContext(
-          fileStatus,
-          inputPolicy,
-          changeDetectionPolicy,
-          readAhead,
-          auditSpan);
-    }
+    final S3AFileStatus fileStatus =
+        trackDuration(inputStreamStats,
+            ACTION_FILE_OPENED.getSymbol(), () ->
+            extractOrFetchSimpleFileStatus(path, fileInformation));
+    S3AReadOpContext readContext = createReadContext(
+        fileStatus,
+        auditSpan);
+    fileInformation.applyOptions(readContext);
     LOG.debug("Opening '{}'", readContext);
-
     return new FSDataInputStream(
         new S3AInputStream(
-            readContext,
-            createObjectAttributes(fileStatus),
-            createInputStreamCallbacks(auditSpan)));
+            readContext.build(),
+            createObjectAttributes(path, fileStatus),
+            createInputStreamCallbacks(auditSpan),
+            inputStreamStats));
   }
 
   /**
@@ -1503,34 +1501,40 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         return s3.getObject(request);
       }
     }
+
+    @Override
+    public <T> CompletableFuture<T> submit(final CallableRaisingIOE<T> operation) {
+      CompletableFuture<T> result = new CompletableFuture<>();
+      unboundedThreadPool.submit(() ->
+          LambdaUtils.eval(result, () -> {
+            try (AuditSpan span = auditSpan.activate()) {
+              return operation.apply();
+            }
+          }));
+      return result;
+    }
   }
 
   /**
    * Create the read context for reading from the referenced file,
    * using FS state as well as the status.
    * @param fileStatus file status.
-   * @param seekPolicy input policy for this operation
-   * @param changePolicy change policy for this operation.
-   * @param readAheadRange readahead value.
    * @param auditSpan audit span.
    * @return a context for read and select operations.
    */
   @VisibleForTesting
   protected S3AReadOpContext createReadContext(
       final FileStatus fileStatus,
-      final S3AInputPolicy seekPolicy,
-      final ChangeDetectionPolicy changePolicy,
-      final long readAheadRange,
       final AuditSpan auditSpan) {
-    return new S3AReadOpContext(fileStatus.getPath(),
+    final S3AReadOpContext roc = new S3AReadOpContext(
+        fileStatus.getPath(),
         invoker,
         statistics,
         statisticsContext,
-        fileStatus,
-        seekPolicy,
-        changePolicy,
-        readAheadRange,
-        auditSpan);
+        fileStatus)
+        .withAuditSpan(auditSpan);
+    openFileHelper.applyDefaultOptions(roc);
+    return roc.build();
   }
 
   /**
@@ -1558,13 +1562,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   /**
    * Create the attributes of an object for subsequent use.
+   * @param path path -this is used over the file status path.
    * @param fileStatus file status to build from.
    * @return attributes to use when building the query.
    */
   private S3ObjectAttributes createObjectAttributes(
+      final Path path,
       final S3AFileStatus fileStatus) {
     return createObjectAttributes(
-        fileStatus.getPath(),
+        path,
         fileStatus.getEtag(),
         fileStatus.getVersionId(),
         fileStatus.getLen());
@@ -1981,14 +1987,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     @Override
     public S3ObjectAttributes createObjectAttributes(
         final S3AFileStatus fileStatus) {
-      return S3AFileSystem.this.createObjectAttributes(fileStatus);
+      return S3AFileSystem.this.createObjectAttributes(
+          fileStatus.getPath(),
+          fileStatus);
     }
 
     @Override
     public S3AReadOpContext createReadContext(final FileStatus fileStatus) {
       return S3AFileSystem.this.createReadContext(fileStatus,
-          inputPolicy,
-          changeDetectionPolicy, readAhead,
           auditSpan);
     }
 
@@ -4085,9 +4091,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   /**
    * Return the number of bytes that large input files should be optimally
    * be split into to minimize I/O time.
-   * @deprecated use {@link #getDefaultBlockSize(Path)} instead
    */
-  @Deprecated
   public long getDefaultBlockSize() {
     return getConf().getLongBytes(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE);
   }
@@ -4106,14 +4110,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         "S3AFileSystem{");
     sb.append("uri=").append(uri);
     sb.append(", workingDir=").append(workingDir);
-    sb.append(", inputPolicy=").append(inputPolicy);
     sb.append(", partSize=").append(partSize);
     sb.append(", enableMultiObjectsDelete=").append(enableMultiObjectsDelete);
     sb.append(", maxKeys=").append(maxKeys);
     if (cannedACL != null) {
-      sb.append(", cannedACL=").append(cannedACL.toString());
+      sb.append(", cannedACL=").append(cannedACL);
+    }
+    if (openFileHelper != null) {
+      sb.append(", ").append(openFileHelper);
     }
-    sb.append(", readAhead=").append(readAhead);
     if (getConf() != null) {
       sb.append(", blockSize=").append(getDefaultBlockSize());
     }
@@ -4799,23 +4804,26 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @Retries.RetryTranslated
   @AuditEntryPoint
   private FSDataInputStream select(final Path source,
-      final String expression,
       final Configuration options,
-      final Optional<S3AFileStatus> providedStatus)
+      final OpenFileSupport.OpenFileInformation fileInformation)
       throws IOException {
-    final AuditSpan auditSpan = entryPoint(OBJECT_SELECT_REQUESTS, source);
     requireSelectSupport(source);
+    final AuditSpan auditSpan = entryPoint(OBJECT_SELECT_REQUESTS, source);
     final Path path = makeQualified(source);
+    String expression = fileInformation.getSql();
     final S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path,
-        providedStatus);
+        fileInformation);
 
     // readahead range can be dynamically set
-    long ra = options.getLong(READAHEAD_RANGE, readAhead);
-    S3ObjectAttributes objectAttributes = createObjectAttributes(fileStatus);
-    S3AReadOpContext readContext = createReadContext(fileStatus, inputPolicy,
-        changeDetectionPolicy, ra, auditSpan);
+    S3ObjectAttributes objectAttributes = createObjectAttributes(
+        path, fileStatus);
+    ChangeDetectionPolicy changePolicy = fileInformation.getChangePolicy();
+    S3AReadOpContext readContext = createReadContext(
+        fileStatus,
+        auditSpan);
+    fileInformation.applyOptions(readContext);
 
-    if (changeDetectionPolicy.getSource() != ChangeDetectionPolicy.Source.None
+    if (changePolicy.getSource() != ChangeDetectionPolicy.Source.None
         && fileStatus.getEtag() != null) {
       // if there is change detection, and the status includes at least an
       // etag,
@@ -4827,7 +4835,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       // version in the final read; nor can we check the etag match)
       ChangeTracker changeTracker =
           new ChangeTracker(uri.toString(),
-              changeDetectionPolicy,
+              changePolicy,
               readContext.getS3AStatisticsContext()
                   .newInputStreamStatistics()
                   .getChangeTrackerStatistics(),
@@ -4865,38 +4873,36 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
-   * Extract the status from the optional parameter, querying
-   * S3 if it is absent.
-   * @param path path of the status
-   * @param optStatus optional status
+   * Get the file status of the source file.
+   * If in the fileInformation parameter return that
+   * if not found, issue a HEAD request, looking for a
+   * file only.
+   * @param path path of the file to open
+   * @param fileInformation information on the file to open
    * @return a file status
-   * @throws FileNotFoundException if there is no normal file at that path
+   * @throws FileNotFoundException if a HEAD request found no file
    * @throws IOException IO failure
    */
   private S3AFileStatus extractOrFetchSimpleFileStatus(
-      final Path path, final Optional<S3AFileStatus> optStatus)
+      final Path path,
+      final OpenFileSupport.OpenFileInformation fileInformation)
       throws IOException {
-    S3AFileStatus fileStatus;
-    if (optStatus.isPresent()) {
-      fileStatus = optStatus.get();
+    S3AFileStatus fileStatus = fileInformation.getStatus();
+    if (fileStatus == null) {
       // we check here for the passed in status
       // being a directory
-      if (fileStatus.isDirectory()) {
-        throw new FileNotFoundException(path.toString() + " is a directory");
-      }
-    } else {
-      // Executes a HEAD only.
-      // therefore: if there is is a dir marker, this
-      // will raise a FileNotFoundException
       fileStatus = innerGetFileStatus(path, false,
           StatusProbeEnum.HEAD_ONLY);
     }
+    if (fileStatus.isDirectory()) {
+      throw new FileNotFoundException(path.toString() + " is a directory");
+    }
 
     return fileStatus;
   }
 
   /**
-   * Initiate the open or select operation.
+   * Initiate the open() or select() operation.
    * This is invoked from both the FileSystem and FileContext APIs.
    * It's declared as an audit entry point but the span creation is pushed
    * down into the open/select methods it ultimately calls.
@@ -4915,54 +4921,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       final Path rawPath,
       final OpenFileParameters parameters) throws IOException {
     final Path path = qualify(rawPath);
-    Configuration options = parameters.getOptions();
-    Set<String> mandatoryKeys = parameters.getMandatoryKeys();
-    String sql = options.get(SelectConstants.SELECT_SQL, null);
-    boolean isSelect = sql != null;
-    // choice of keys depends on open type
-    if (isSelect) {
-      rejectUnknownMandatoryKeys(
-          mandatoryKeys,
-          InternalSelectConstants.SELECT_OPTIONS,
-          "for " + path + " in S3 Select operation");
-    } else {
-      rejectUnknownMandatoryKeys(
-          mandatoryKeys,
-          InternalConstants.STANDARD_OPENFILE_KEYS,
-          "for " + path + " in non-select file I/O");
-    }
-    FileStatus providedStatus = parameters.getStatus();
-    S3AFileStatus fileStatus;
-    if (providedStatus != null) {
-      Preconditions.checkArgument(path.equals(providedStatus.getPath()),
-          "FileStatus parameter is not for the path %s: %s",
-          path, providedStatus);
-      if (providedStatus instanceof S3AFileStatus) {
-        // can use this status to skip our own probes,
-        // including etag and version.
-        LOG.debug("File was opened with a supplied S3AFileStatus;"
-            + " skipping getFileStatus call in open() operation: {}",
-            providedStatus);
-        fileStatus = (S3AFileStatus) providedStatus;
-      } else if (providedStatus instanceof S3ALocatedFileStatus) {
-        LOG.debug("File was opened with a supplied S3ALocatedFileStatus;"
-            + " skipping getFileStatus call in open() operation: {}",
-            providedStatus);
-        fileStatus = ((S3ALocatedFileStatus) providedStatus).toS3AFileStatus();
-      } else {
-        LOG.debug("Ignoring file status {}", providedStatus);
-        fileStatus = null;
-      }
-    } else {
-      fileStatus = null;
-    }
-    Optional<S3AFileStatus> ost = Optional.ofNullable(fileStatus);
+    OpenFileSupport.OpenFileInformation fileInformation =
+        openFileHelper.prepareToOpenFile(
+            path,
+            parameters,
+            getDefaultBlockSize());
     CompletableFuture<FSDataInputStream> result = new CompletableFuture<>();
-    if (!isSelect) {
+    if (!fileInformation.isS3Select()) {
       // normal path.
       unboundedThreadPool.submit(() ->
           LambdaUtils.eval(result,
-              () -> open(path, Optional.of(options), ost)));
+              () -> executeOpen(path, fileInformation)));
     } else {
       // it is a select statement.
       // fail fast if the operation is not available
@@ -4970,7 +4939,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       // submit the query
       unboundedThreadPool.submit(() ->
           LambdaUtils.eval(result,
-              () -> select(path, sql, options, ost)));
+              () -> select(path, parameters.getOptions(), fileInformation)));
     }
     return result;
   }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java
index c018410e117..b90d0f2a616 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java
@@ -18,32 +18,46 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import javax.annotation.Nullable;
+import java.util.Collection;
+import java.util.Locale;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Locale;
 
-import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_VECTOR;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
 
 /**
- * Filesystem input policy.
+ * Stream input policy.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public enum S3AInputPolicy {
 
-  Normal(INPUT_FADV_NORMAL),
-  Sequential(INPUT_FADV_SEQUENTIAL),
-  Random(INPUT_FADV_RANDOM);
+  Normal(FS_OPTION_OPENFILE_READ_POLICY_DEFAULT, false, true),
+  Random(FS_OPTION_OPENFILE_READ_POLICY_RANDOM, true, false),
+  Sequential(FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL, false, false);
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(S3AInputPolicy.class);
+  /** Policy name. */
   private final String policy;
 
-  S3AInputPolicy(String policy) {
+  /** Is this random IO? */
+  private final boolean randomIO;
+
+  /** Is this an adaptive policy? */
+  private final boolean adaptive;
+
+  S3AInputPolicy(String policy,
+      boolean randomIO,
+      boolean adaptive) {
     this.policy = policy;
+    this.randomIO = randomIO;
+    this.adaptive = adaptive;
   }
 
   @Override
@@ -51,26 +65,63 @@ public enum S3AInputPolicy {
     return policy;
   }
 
+  String getPolicy() {
+    return policy;
+  }
+
+  boolean isRandomIO() {
+    return randomIO;
+  }
+
+  boolean isAdaptive() {
+    return adaptive;
+  }
+
   /**
-   * Choose an FS access policy.
-   * Always returns something,
-   * primarily by downgrading to "normal" if there is no other match.
+   * Choose an access policy.
    * @param name strategy name from a configuration option, etc.
+   * @param defaultPolicy default policy to fall back to.
    * @return the chosen strategy
    */
-  public static S3AInputPolicy getPolicy(String name) {
+  public static S3AInputPolicy getPolicy(
+      String name,
+      @Nullable S3AInputPolicy defaultPolicy) {
     String trimmed = name.trim().toLowerCase(Locale.ENGLISH);
     switch (trimmed) {
-    case INPUT_FADV_NORMAL:
+    case FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE:
+    case FS_OPTION_OPENFILE_READ_POLICY_DEFAULT:
+    case Constants.INPUT_FADV_NORMAL:
       return Normal;
-    case INPUT_FADV_RANDOM:
+
+    // all these options currently map to random IO.
+    case FS_OPTION_OPENFILE_READ_POLICY_RANDOM:
+    case FS_OPTION_OPENFILE_READ_POLICY_VECTOR:
       return Random;
-    case INPUT_FADV_SEQUENTIAL:
+
+    case FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL:
+    case FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE:
       return Sequential;
     default:
-      LOG.warn("Unrecognized " + INPUT_FADVISE + " value: \"{}\"", trimmed);
-      return Normal;
+      return defaultPolicy;
+    }
+  }
+
+  /**
+   * Scan the list of input policies, returning the first one supported.
+   * @param policies list of policies.
+   * @param defaultPolicy fallback
+   * @return a policy or the defaultPolicy, which may be null
+   */
+  public static S3AInputPolicy getFirstSupportedPolicy(
+      Collection<String> policies,
+      @Nullable S3AInputPolicy defaultPolicy) {
+    for (String s : policies) {
+      S3AInputPolicy nextPolicy = S3AInputPolicy.getPolicy(s, null);
+      if (nextPolicy != null) {
+        return nextPolicy;
+      }
     }
+    return defaultPolicy;
   }
 
 }
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 79a65acb438..6beeb2891ee 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
@@ -23,6 +23,7 @@ import javax.annotation.Nullable;
 import com.amazonaws.services.s3.model.GetObjectRequest;
 import com.amazonaws.services.s3.model.S3Object;
 import com.amazonaws.services.s3.model.S3ObjectInputStream;
+
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -37,7 +38,7 @@ 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.fs.statistics.DurationTracker;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,9 +47,14 @@ 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.s3a.Invoker.onceTrackingDuration;
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
 import static org.apache.hadoop.util.StringUtils.toLowerCase;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /**
  * The input stream for an S3A object.
@@ -78,6 +84,11 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   public static final String OPERATION_OPEN = "open";
   public static final String OPERATION_REOPEN = "re-open";
 
+  /**
+   * size of a buffer to create when draining the stream.
+   */
+  private static final int DRAIN_BUFFER_SIZE = 16384;
+
   /**
    * This is the public position; the one set in {@link #seek(long)}
    * and returned in {@link #getPos()}.
@@ -136,6 +147,12 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    */
   private final IOStatistics ioStatistics;
 
+  /**
+   * Threshold for stream reads to switch to
+   * asynchronous draining.
+   */
+  private long asyncDrainThreshold;
+
   /**
    * Create the stream.
    * This does not attempt to open it; that is only done on the first
@@ -143,10 +160,12 @@ 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
    */
   public S3AInputStream(S3AReadOpContext ctx,
       S3ObjectAttributes s3Attributes,
-      InputStreamCallbacks client) {
+      InputStreamCallbacks client,
+      S3AInputStreamStatistics streamStatistics) {
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()),
         "No Bucket");
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key");
@@ -155,12 +174,11 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     this.context = ctx;
     this.bucket = s3Attributes.getBucket();
     this.key = s3Attributes.getKey();
-    this.pathStr = ctx.dstFileStatus.getPath().toString();
+    this.pathStr = s3Attributes.getPath().toString();
     this.contentLength = l;
     this.client = client;
     this.uri = "s3a://" + this.bucket + "/" + this.key;
-    this.streamStatistics = ctx.getS3AStatisticsContext()
-        .newInputStreamStatistics();
+    this.streamStatistics = streamStatistics;
     this.ioStatistics = streamStatistics.getIOStatistics();
     this.changeTracker = new ChangeTracker(uri,
         ctx.getChangeDetectionPolicy(),
@@ -168,6 +186,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
         s3Attributes);
     setInputPolicy(ctx.getInputPolicy());
     setReadahead(ctx.getReadahead());
+    this.asyncDrainThreshold = ctx.getAsyncDrainThreshold();
   }
 
   /**
@@ -193,7 +212,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
           boolean forceAbort) throws IOException {
 
     if (isObjectStreamOpen()) {
-      closeStream("reopen(" + reason + ")", contentRangeFinish, forceAbort);
+      closeStream("reopen(" + reason + ")", forceAbort, false);
     }
 
     contentRangeFinish = calculateRequestLimit(inputPolicy, targetPos,
@@ -211,21 +230,10 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
         operation, uri, targetPos);
     changeTracker.maybeApplyConstraint(request);
 
-    DurationTracker tracker = streamStatistics.initiateGetRequest();
-    try {
-      object = Invoker.once(text, uri,
-          () -> client.getObject(request));
-    } catch(IOException e) {
-      // input function failed: note it
-      tracker.failed();
-      // and rethrow
-      throw e;
-    } finally {
-      // update the tracker.
-      // this is called after any catch() call will have
-      // set the failed flag.
-      tracker.close();
-    }
+    object = onceTrackingDuration(text, uri,
+        streamStatistics.initiateGetRequest(), () ->
+            client.getObject(request));
+
 
     changeTracker.processResponse(object, operation,
         targetPos);
@@ -333,7 +341,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
       streamStatistics.seekBackwards(diff);
       // if the stream is in "Normal" mode, switch to random IO at this
       // point, as it is indicative of columnar format IO
-      if (inputPolicy.equals(S3AInputPolicy.Normal)) {
+      if (inputPolicy.isAdaptive()) {
         LOG.info("Switching to Random IO seek policy");
         setInputPolicy(S3AInputPolicy.Random);
       }
@@ -348,7 +356,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
 
     // if the code reaches here, the stream needs to be reopened.
     // close the stream; if read the object will be opened at the new pos
-    closeStream("seekInStream()", this.contentRangeFinish, false);
+    closeStream("seekInStream()", false, false);
     pos = targetPos;
   }
 
@@ -458,7 +466,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
           uri, client, object);
     }
     streamStatistics.readException();
-    closeStream("failure recovery", contentRangeFinish, forceAbort);
+    closeStream("failure recovery", forceAbort, false);
   }
 
   /**
@@ -551,8 +559,8 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     if (!closed) {
       closed = true;
       try {
-        // close or abort the stream
-        closeStream("close() operation", this.contentRangeFinish, false);
+        // close or abort the stream; blocking
+        awaitFuture(closeStream("close() operation", false, true));
         LOG.debug("Statistics of stream {}\n{}", key, streamStatistics);
         // end the client+audit span.
         client.close();
@@ -571,16 +579,26 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    * If a close() is attempted and fails, the operation escalates to
    * an abort.
    *
+   * The close is potentially; a future is returned.
+   * It's the draining of a stream which is time consuming so
+   * worth scheduling on a separate thread.
+   * In stream close, when an abort is issued or when there's no
+   * data to drain, block.
    * This does not set the {@link #closed} flag.
    * @param reason reason for stream being closed; used in messages
-   * @param length length of the stream.
    * @param forceAbort force an abort; used if explicitly requested.
+   * @param blocking should the call block for completion, or is async IO allowed
+   * @return a future for the async operation
    */
   @Retries.OnceRaw
-  private void closeStream(String reason, long length, boolean forceAbort) {
+  private CompletableFuture<Boolean> closeStream(
+      final String reason,
+      final boolean forceAbort,
+      final boolean blocking) {
+
     if (!isObjectStreamOpen()) {
       // steam is already closed
-      return;
+      return CompletableFuture.completedFuture(false);
     }
 
     // if the amount of data remaining in the current request is greater
@@ -589,56 +607,137 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     LOG.debug("Closing stream {}: {}", reason,
         forceAbort ? "abort" : "soft");
     boolean shouldAbort = forceAbort || remaining > readahead;
+    CompletableFuture<Boolean> operation;
+
+    if (blocking || shouldAbort || remaining <= asyncDrainThreshold) {
+      // don't bother with async io.
+      operation = CompletableFuture.completedFuture(
+          drain(shouldAbort, reason, remaining, object, wrappedStream));
+
+    } else {
+      LOG.debug("initiating asynchronous drain of {} bytes", remaining);
+      // schedule an async drain/abort with references to the fields so they
+      // can be reused
+      operation = client.submit(
+          () -> drain(false, reason, remaining, object, wrappedStream));
+    }
+
+    // either the stream is closed in the blocking call or the async call is
+    // submitted with its own copy of the references
+    wrappedStream = null;
+    object = null;
+    return operation;
+  }
+
+  /**
+   * drain the stream. This method is intended to be
+   * used directly or asynchronously, and measures the
+   * duration of the operation in the stream statistics.
+   * @param shouldAbort force an abort; used if explicitly requested.
+   * @param reason reason for stream being closed; used in messages
+   * @param remaining remaining bytes
+   * @param requestObject http request object; needed to avoid GC issues.
+   * @param inner stream to close.
+   * @return was the stream aborted?
+   */
+  private boolean drain(
+      final boolean shouldAbort,
+      final String reason,
+      final long remaining,
+      final S3Object requestObject,
+      final S3ObjectInputStream inner) {
 
     try {
-      if (!shouldAbort) {
-        try {
-          // clean close. This will read to the end of the stream,
-          // so, while cleaner, can be pathological on a multi-GB object
-
-          // explicitly drain the stream
-          long drained = 0;
-          while (wrappedStream.read() >= 0) {
-            drained++;
+      return invokeTrackingDuration(
+          streamStatistics.initiateInnerStreamClose(shouldAbort),
+          () -> drainOrAbortHttpStream(
+              shouldAbort,
+              reason,
+              remaining,
+              requestObject,
+              inner));
+    } catch (IOException e) {
+      // this is only here because invokeTrackingDuration() has it in its
+      // signature
+      return shouldAbort;
+    }
+  }
+
+  /**
+   * Drain or abort the inner stream.
+   * Exceptions are swallowed.
+   * If a close() is attempted and fails, the operation escalates to
+   * an abort.
+   *
+   * This does not set the {@link #closed} flag.
+   *
+   * A reference to the stream is passed in so that the instance
+   * {@link #wrappedStream} field can be reused as soon as this
+   * method is submitted;
+   * @param shouldAbort force an abort; used if explicitly requested.
+   * @param reason reason for stream being closed; used in messages
+   * @param remaining remaining bytes
+   * @param requestObject http request object; needed to avoid GC issues.
+   * @param inner stream to close.
+   * @return was the stream aborted?
+   */
+  private boolean drainOrAbortHttpStream(
+      boolean shouldAbort,
+      final String reason,
+      final long remaining,
+      final S3Object requestObject,
+      final S3ObjectInputStream inner) {
+    // force a use of the request object so IDEs don't warn of
+    // lack of use.
+    requireNonNull(requestObject);
+
+    if (!shouldAbort) {
+      try {
+        // clean close. This will read to the end of the stream,
+        // so, while cleaner, can be pathological on a multi-GB object
+
+        // explicitly drain the stream
+        long drained = 0;
+        byte[] buffer = new byte[DRAIN_BUFFER_SIZE];
+        while (true) {
+          final int count = inner.read(buffer);
+          if (count < 0) {
+            // no more data is left
+            break;
           }
-          LOG.debug("Drained stream of {} bytes", drained);
-
-          // now close it
-          wrappedStream.close();
-          // this MUST come after the close, so that if the IO operations fail
-          // and an abort is triggered, the initial attempt's statistics
-          // aren't collected.
-          streamStatistics.streamClose(false, drained);
-        } catch (Exception e) {
-          // exception escalates to an abort
-          LOG.debug("When closing {} stream for {}, will abort the stream",
-              uri, reason, e);
-          shouldAbort = true;
+          drained += count;
         }
+        LOG.debug("Drained stream of {} bytes", drained);
+
+        // now close it
+        inner.close();
+        // this MUST come after the close, so that if the IO operations fail
+        // and an abort is triggered, the initial attempt's statistics
+        // aren't collected.
+        streamStatistics.streamClose(false, drained);
+      } catch (Exception e) {
+        // exception escalates to an abort
+        LOG.debug("When closing {} stream for {}, will abort the stream",
+            uri, reason, e);
+        shouldAbort = true;
       }
-      if (shouldAbort) {
-        // Abort, rather than just close, the underlying stream.  Otherwise, the
-        // remaining object payload is read from S3 while closing the stream.
-        LOG.debug("Aborting stream {}", uri);
-        try {
-          wrappedStream.abort();
-        } catch (Exception e) {
-          LOG.warn("When aborting {} stream after failing to close it for {}",
-              uri, reason, e);
-        }
-        streamStatistics.streamClose(true, remaining);
+    }
+    if (shouldAbort) {
+      // Abort, rather than just close, the underlying stream.  Otherwise, the
+      // remaining object payload is read from S3 while closing the stream.
+      LOG.debug("Aborting stream {}", uri);
+      try {
+        inner.abort();
+      } catch (Exception e) {
+        LOG.warn("When aborting {} stream after failing to close it for {}",
+            uri, reason, e);
       }
-      LOG.debug("Stream {} {}: {}; remaining={} streamPos={},"
-              + " nextReadPos={}," +
-              " request range {}-{} length={}",
-          uri, (shouldAbort ? "aborted" : "closed"), reason,
-          remaining, pos, nextReadPos,
-          contentRangeStart, contentRangeFinish,
-          length);
-    } finally {
-      wrappedStream = null;
-      object = null;
+      streamStatistics.streamClose(true, remaining);
     }
+    LOG.debug("Stream {} {}: {}; remaining={}",
+        uri, (shouldAbort ? "aborted" : "closed"), reason,
+        remaining);
+    return shouldAbort;
   }
 
   /**
@@ -648,17 +747,17 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
    *
    * This is potentially very inefficient, and should only be invoked
    * in extreme circumstances. It logs at info for this reason.
+   *
+   * Blocks until the abort is completed.
+   *
    * @return true if the connection was actually reset.
    * @throws IOException if invoked on a closed stream.
    */
   @InterfaceStability.Unstable
   public synchronized boolean resetConnection() throws IOException {
     checkNotClosed();
-    if (isObjectStreamOpen()) {
-      LOG.info("Forced reset of connection to {}", uri);
-      closeStream("reset()", contentRangeFinish, true);
-    }
-    return isObjectStreamOpen();
+    LOG.info("Forcing reset of connection to {}", uri);
+    return awaitFuture(closeStream("reset()", true, true));
   }
 
   @Override
@@ -870,7 +969,7 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
   @Override
   public synchronized void unbuffer() {
     try {
-      closeStream("unbuffer()", contentRangeFinish, false);
+      closeStream("unbuffer()", false, false);
     } finally {
       streamStatistics.unbuffered();
     }
@@ -918,6 +1017,14 @@ public class S3AInputStream extends FSInputStream implements  CanSetReadahead,
     @Retries.OnceRaw
     S3Object getObject(GetObjectRequest request);
 
+    /**
+     * Submit some asynchronous work, for example, draining a stream.
+     * @param operation operation to invoke
+     * @param <T> return type
+     * @return a future.
+     */
+    <T> CompletableFuture<T> submit(CallableRaisingIOE<T> operation);
+
   }
 
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index 099d6442ca0..eec63667201 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
 import org.apache.hadoop.fs.statistics.IOStatisticsLogging;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+import org.apache.hadoop.fs.statistics.StoreStatisticNames;
 import org.apache.hadoop.fs.statistics.StreamStatisticNames;
 import org.apache.hadoop.fs.statistics.DurationTracker;
 import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
@@ -837,7 +838,10 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
               StreamStatisticNames.STREAM_READ_UNBUFFERED,
               StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES)
           .withGauges(STREAM_READ_GAUGE_INPUT_POLICY)
-          .withDurationTracking(ACTION_HTTP_GET_REQUEST)
+          .withDurationTracking(ACTION_HTTP_GET_REQUEST,
+              StoreStatisticNames.ACTION_FILE_OPENED,
+              StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED,
+              StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED)
           .build();
       setIOStatistics(st);
       aborted = st.getCounterReference(
@@ -1271,6 +1275,12 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
       return trackDuration(ACTION_HTTP_GET_REQUEST);
     }
 
+    @Override
+    public DurationTracker initiateInnerStreamClose(final boolean abort) {
+      return trackDuration(abort
+          ? StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED
+          : StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED);
+    }
   }
 
   /**
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 5ce3d96c435..f416cf9485d 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
@@ -29,7 +29,7 @@ import javax.annotation.Nullable;
 
 import org.apache.hadoop.util.Preconditions;
 
-import static org.apache.hadoop.util.Preconditions.checkNotNull;
+import static java.util.Objects.requireNonNull;
 
 /**
  * Read-specific operation context struct.
@@ -44,19 +44,25 @@ public class S3AReadOpContext extends S3AOpContext {
   /**
    * Initial input policy of the stream.
    */
-  private final S3AInputPolicy inputPolicy;
+  private S3AInputPolicy inputPolicy;
 
   /**
    * How to detect and deal with the object being updated during read.
    */
-  private final ChangeDetectionPolicy changeDetectionPolicy;
+  private ChangeDetectionPolicy changeDetectionPolicy;
 
   /**
    * Readahead for GET operations/skip, etc.
    */
-  private final long readahead;
+  private long readahead;
 
-  private final AuditSpan auditSpan;
+  private AuditSpan auditSpan;
+
+  /**
+   * Threshold for stream reads to switch to
+   * asynchronous draining.
+   */
+  private long asyncDrainThreshold;
 
   /**
    * Instantiate.
@@ -65,31 +71,33 @@ public class S3AReadOpContext extends S3AOpContext {
    * @param stats Fileystem statistics (may be null)
    * @param instrumentation statistics context
    * @param dstFileStatus target file status
-   * @param inputPolicy the input policy
-   * @param changeDetectionPolicy change detection policy.
-   * @param readahead readahead for GET operations/skip, etc.
-   * @param auditSpan active audit
    */
   public S3AReadOpContext(
       final Path path,
       Invoker invoker,
       @Nullable FileSystem.Statistics stats,
       S3AStatisticsContext instrumentation,
-      FileStatus dstFileStatus,
-      S3AInputPolicy inputPolicy,
-      ChangeDetectionPolicy changeDetectionPolicy,
-      final long readahead,
-      final AuditSpan auditSpan) {
+      FileStatus dstFileStatus) {
 
     super(invoker, stats, instrumentation,
         dstFileStatus);
-    this.path = checkNotNull(path);
-    this.auditSpan = auditSpan;
+    this.path = requireNonNull(path);
+  }
+
+  /**
+   * validate the context.
+   * @return a read operation context ready for use.
+   */
+  public S3AReadOpContext build() {
+    requireNonNull(inputPolicy, "inputPolicy");
+    requireNonNull(changeDetectionPolicy, "changeDetectionPolicy");
+    requireNonNull(auditSpan, "auditSpan");
+    requireNonNull(inputPolicy, "inputPolicy");
     Preconditions.checkArgument(readahead >= 0,
         "invalid readahead %d", readahead);
-    this.inputPolicy = checkNotNull(inputPolicy);
-    this.changeDetectionPolicy = checkNotNull(changeDetectionPolicy);
-    this.readahead = readahead;
+    Preconditions.checkArgument(asyncDrainThreshold >= 0,
+        "invalid drainThreshold %d", asyncDrainThreshold);
+    return this;
   }
 
   /**
@@ -136,6 +144,61 @@ public class S3AReadOpContext extends S3AOpContext {
     return auditSpan;
   }
 
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return the builder
+   */
+  public S3AReadOpContext withInputPolicy(final S3AInputPolicy value) {
+    inputPolicy = value;
+    return this;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return the builder
+   */
+  public S3AReadOpContext withChangeDetectionPolicy(
+      final ChangeDetectionPolicy value) {
+    changeDetectionPolicy = value;
+    return this;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return the builder
+   */
+  public S3AReadOpContext withReadahead(final long value) {
+    readahead = value;
+    return this;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return the builder
+   */
+  public S3AReadOpContext withAuditSpan(final AuditSpan value) {
+    auditSpan = value;
+    return this;
+  }
+
+  /**
+   * Set builder value.
+   * @param value new value
+   * @return the builder
+   */
+  public S3AReadOpContext withAsyncDrainThreshold(final long value) {
+    asyncDrainThreshold = value;
+    return this;
+  }
+
+  public long getAsyncDrainThreshold() {
+    return asyncDrainThreshold;
+  }
+
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder(
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java
index 275b207cd6d..0a0454854b2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
 
 /**
- * This class holds attributed of an object independent of the
+ * This class holds attributes of an object independent of the
  * file status type.
  * It is used in {@link S3AInputStream} and the select equivalent.
  * as a way to reduce parameters being passed
@@ -44,6 +44,17 @@ public class S3ObjectAttributes {
   private final String versionId;
   private final long len;
 
+  /**
+   * Constructor.
+   * @param bucket s3 bucket
+   * @param path path
+   * @param key object key
+   * @param serverSideEncryptionAlgorithm current encryption algorithm
+   * @param serverSideEncryptionKey any server side encryption key?
+   * @param len object length
+   * @param eTag optional etag
+   * @param versionId optional version id
+   */
   public S3ObjectAttributes(
       String bucket,
       Path path,
@@ -70,7 +81,7 @@ public class S3ObjectAttributes {
    * @param copyResult copy result.
    * @param serverSideEncryptionAlgorithm current encryption algorithm
    * @param serverSideEncryptionKey any server side encryption key?
-   * @param len
+   * @param len object length
    */
   public S3ObjectAttributes(
       final Path path,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index ed16a7c4fd8..86cb18076cc 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -55,6 +55,10 @@ public enum Statistic {
       StoreStatisticNames.ACTION_HTTP_HEAD_REQUEST,
       "HEAD request.",
       TYPE_DURATION),
+  ACTION_FILE_OPENED(
+      StoreStatisticNames.ACTION_FILE_OPENED,
+      "File opened.",
+      TYPE_DURATION),
   ACTION_HTTP_GET_REQUEST(
       StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
       "GET request.",
@@ -175,6 +179,10 @@ public enum Statistic {
       StoreStatisticNames.OP_OPEN,
       "Calls of open()",
       TYPE_COUNTER),
+  INVOCATION_OPENFILE(
+      StoreStatisticNames.OP_OPENFILE,
+      "Calls of openFile()",
+      TYPE_COUNTER),
   INVOCATION_RENAME(
       StoreStatisticNames.OP_RENAME,
       "Calls of rename()",
@@ -296,6 +304,15 @@ public enum Statistic {
       StreamStatisticNames.STREAM_READ_OPERATIONS,
       "Count of read() operations in an input stream",
       TYPE_COUNTER),
+  STREAM_READ_REMOTE_STREAM_ABORTED(
+      StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED,
+      "Duration of aborting a remote stream during stream IO",
+      TYPE_DURATION),
+  STREAM_READ_REMOTE_STREAM_CLOSED(
+      StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED,
+      "Duration of closing a remote stream during stream IO",
+      TYPE_DURATION),
+
   STREAM_READ_OPERATIONS_INCOMPLETE(
       StreamStatisticNames.STREAM_READ_OPERATIONS_INCOMPLETE,
       "Count of incomplete read() operations in an input stream",
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
index 0ca71f21aae..840cf8e0f23 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
@@ -267,7 +267,7 @@ public class CommitOperations extends AbstractStoreOperation
     List<Pair<LocatedFileStatus, IOException>> failures = new ArrayList<>(1);
     for (LocatedFileStatus status : statusList) {
       try {
-        commits.add(SinglePendingCommit.load(fs, status.getPath()));
+        commits.add(SinglePendingCommit.load(fs, status.getPath(), status));
       } catch (IOException e) {
         LOG.warn("Failed to load commit file {}", status.getPath(), e);
         failures.add(Pair.of(status, e));
@@ -350,10 +350,12 @@ public class CommitOperations extends AbstractStoreOperation
       LOG.debug("No files to abort under {}", pendingDir);
     }
     while (pendingFiles.hasNext()) {
-      Path pendingFile = pendingFiles.next().getPath();
+      LocatedFileStatus status = pendingFiles.next();
+      Path pendingFile = status.getPath();
       if (pendingFile.getName().endsWith(CommitConstants.PENDING_SUFFIX)) {
         try {
-          abortSingleCommit(SinglePendingCommit.load(fs, pendingFile));
+          abortSingleCommit(SinglePendingCommit.load(fs, pendingFile,
+              status));
         } catch (FileNotFoundException e) {
           LOG.debug("listed file already deleted: {}", pendingFile);
         } catch (IOException | IllegalArgumentException e) {
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java
index fd734102566..318896e2361 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a.commit.files;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.util.ArrayList;
@@ -128,10 +129,7 @@ public class PendingSet extends PersistentCommitData
    */
   public static PendingSet load(FileSystem fs, Path path)
       throws IOException {
-    LOG.debug("Reading pending commits in file {}", path);
-    PendingSet instance = serializer().load(fs, path);
-    instance.validate();
-    return instance;
+    return load(fs, path, null);
   }
 
   /**
@@ -144,7 +142,25 @@ public class PendingSet extends PersistentCommitData
    */
   public static PendingSet load(FileSystem fs, FileStatus status)
       throws IOException {
-    return load(fs, status.getPath());
+    return load(fs, status.getPath(), status);
+  }
+
+  /**
+   * Load an instance from a file, then validate it.
+   * @param fs filesystem
+   * @param path path
+   * @param status status of file to load
+   * @return the loaded instance
+   * @throws IOException IO failure
+   * @throws ValidationFailure if the data is invalid
+   */
+  public static PendingSet load(FileSystem fs, Path path,
+      @Nullable FileStatus status)
+      throws IOException {
+    LOG.debug("Reading pending commits in file {}", path);
+    PendingSet instance = serializer().load(fs, path, status);
+    instance.validate();
+    return instance;
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java
index 9bd37f19f2e..b53ef75d823 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a.commit.files;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.Serializable;
@@ -38,6 +39,7 @@ import org.apache.hadoop.util.Preconditions;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.commit.ValidationFailure;
@@ -152,7 +154,23 @@ public class SinglePendingCommit extends PersistentCommitData
    */
   public static SinglePendingCommit load(FileSystem fs, Path path)
       throws IOException {
-    SinglePendingCommit instance = serializer().load(fs, path);
+    return load(fs, path, null);
+  }
+
+  /**
+   * Load an instance from a file, then validate it.
+   * @param fs filesystem
+   * @param path path
+   * @param status status of file to load or null
+   * @return the loaded instance
+   * @throws IOException IO failure
+   * @throws ValidationFailure if the data is invalid
+   */
+  public static SinglePendingCommit load(FileSystem fs,
+      Path path,
+      @Nullable FileStatus status)
+      throws IOException {
+    SinglePendingCommit instance = serializer().load(fs, path, status);
     instance.filename = path.toString();
     instance.validate();
     return instance;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java
index 75d06e78729..3537755e790 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java
@@ -18,9 +18,9 @@
 
 package org.apache.hadoop.fs.s3a.impl;
 
-import org.apache.hadoop.fs.store.audit.AuditSpan;
+import javax.annotation.Nullable;
 
-import static org.apache.hadoop.util.Preconditions.checkNotNull;
+import org.apache.hadoop.fs.store.audit.AuditSpan;
 
 /**
  * Base class of operations in the store.
@@ -37,7 +37,7 @@ public abstract class AbstractStoreOperation {
   /**
    * Audit Span.
    */
-  private AuditSpan auditSpan;
+  private final AuditSpan auditSpan;
 
   /**
    * Constructor.
@@ -45,8 +45,11 @@ public abstract class AbstractStoreOperation {
    * stores it for later.
    * @param storeContext store context.
    */
-  protected AbstractStoreOperation(final StoreContext storeContext) {
-    this(storeContext, storeContext.getActiveAuditSpan());
+  protected AbstractStoreOperation(final @Nullable StoreContext storeContext) {
+    this(storeContext,
+        storeContext != null
+        ? storeContext.getActiveAuditSpan()
+        : null);
   }
 
   /**
@@ -54,10 +57,11 @@ public abstract class AbstractStoreOperation {
    * @param storeContext store context.
    * @param auditSpan active span
    */
-  protected AbstractStoreOperation(final StoreContext storeContext,
+  protected AbstractStoreOperation(
+      final @Nullable StoreContext storeContext,
       final AuditSpan auditSpan) {
-    this.storeContext = checkNotNull(storeContext);
-    this.auditSpan = checkNotNull(auditSpan);
+    this.storeContext = storeContext;
+    this.auditSpan = auditSpan;
   }
 
   /**
@@ -70,7 +74,7 @@ public abstract class AbstractStoreOperation {
 
   /**
    * Get the audit span this object was created with.
-   * @return the current span
+   * @return the current span or null
    */
   public AuditSpan getAuditSpan() {
     return auditSpan;
@@ -80,6 +84,8 @@ public abstract class AbstractStoreOperation {
    * Activate the audit span.
    */
   public void activateAuditSpan() {
-    auditSpan.activate();
+    if (auditSpan != null) {
+      auditSpan.activate();
+    }
   }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java
index 259738f9989..01562074192 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java
@@ -35,7 +35,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.store.audit.AuditSpan;
 import org.apache.hadoop.util.DurationInfo;
 
-import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause;
+import static org.apache.hadoop.util.functional.FutureIO.raiseInnerCause;
 
 /**
  * A bridge from Callable to Supplier; catching exceptions
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
index 1e4a1262764..49b9feeb6f1 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
@@ -18,16 +18,18 @@
 
 package org.apache.hadoop.fs.s3a.impl;
 
-import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import org.apache.hadoop.classification.VisibleForTesting;
 
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.s3a.Constants;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS;
+
 /**
  * Internal constants private only to the S3A codebase.
  * Please don't refer to these outside of this module &amp; its tests.
@@ -89,11 +91,16 @@ public final class InternalConstants {
    * used becomes that of the select operation.
    */
   @InterfaceStability.Unstable
-  public static final Set<String> STANDARD_OPENFILE_KEYS =
-      Collections.unmodifiableSet(
-          new HashSet<>(
-              Arrays.asList(Constants.INPUT_FADVISE,
-                  Constants.READAHEAD_RANGE)));
+  public static final Set<String> S3A_OPENFILE_KEYS;
+
+  static {
+    Set<String> keys = Stream.of(
+        Constants.INPUT_FADVISE,
+        Constants.READAHEAD_RANGE)
+        .collect(Collectors.toSet());
+    keys.addAll(FS_OPTION_OPENFILE_STANDARD_OPTIONS);
+    S3A_OPENFILE_KEYS = Collections.unmodifiableSet(keys);
+  }
 
   /** 403 error code. */
   public static final int SC_403 = 403;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java
new file mode 100644
index 00000000000..70a99d5318c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java
@@ -0,0 +1,600 @@
+/*
+ * 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.impl;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.impl.OpenFileParameters;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AInputPolicy;
+import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
+import org.apache.hadoop.fs.s3a.S3AReadOpContext;
+import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
+import org.apache.hadoop.fs.s3a.select.SelectConstants;
+
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START;
+import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys;
+import static org.apache.hadoop.fs.s3a.Constants.ASYNC_DRAIN_THRESHOLD;
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE;
+import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE;
+import static org.apache.hadoop.util.Preconditions.checkArgument;
+
+/**
+ * Helper class for openFile() logic, especially processing file status
+ * args and length/etag/versionID.
+ * <p>
+ *  This got complex enough it merited removal from S3AFileSystem -which
+ *  also permits unit testing.
+ * </p>
+ * <p>
+ *   The default values are those from the FileSystem configuration.
+ *   in openFile(), they can all be changed by specific options;
+ *   in FileSystem.open(path, buffersize) only the buffer size is
+ *   set.
+ * </p>
+ */
+public class OpenFileSupport {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OpenFileSupport.class);
+
+  /**
+   * For use when a value of an split/file length is unknown.
+   */
+  private static final int LENGTH_UNKNOWN = -1;
+
+  /**  Default change detection policy. */
+  private final ChangeDetectionPolicy changePolicy;
+
+  /** Default read ahead range. */
+  private final long defaultReadAhead;
+
+  /** Username. */
+  private final String username;
+
+  /** Default buffer size. */
+  private final int defaultBufferSize;
+
+  /**
+   * Threshold for stream reads to switch to
+   * asynchronous draining.
+   */
+  private final long defaultAsyncDrainThreshold;
+
+  /**
+   * Default input policy; may be overridden in
+   * {@code openFile()}.
+   */
+  private final S3AInputPolicy defaultInputPolicy;
+
+  /**
+   * Instantiate with the default options from the filesystem.
+   * @param changePolicy change detection policy
+   * @param defaultReadAhead read ahead range
+   * @param username username
+   * @param defaultBufferSize buffer size
+   * @param defaultAsyncDrainThreshold drain threshold
+   * @param defaultInputPolicy input policy
+   */
+  public OpenFileSupport(
+      final ChangeDetectionPolicy changePolicy,
+      final long defaultReadAhead,
+      final String username,
+      final int defaultBufferSize,
+      final long defaultAsyncDrainThreshold,
+      final S3AInputPolicy defaultInputPolicy) {
+    this.changePolicy = changePolicy;
+    this.defaultReadAhead = defaultReadAhead;
+    this.username = username;
+    this.defaultBufferSize = defaultBufferSize;
+    this.defaultAsyncDrainThreshold = defaultAsyncDrainThreshold;
+    this.defaultInputPolicy = defaultInputPolicy;
+  }
+
+  public ChangeDetectionPolicy getChangePolicy() {
+    return changePolicy;
+  }
+
+  public long getDefaultReadAhead() {
+    return defaultReadAhead;
+  }
+
+  public int getDefaultBufferSize() {
+    return defaultBufferSize;
+  }
+
+  public long getDefaultAsyncDrainThreshold() {
+    return defaultAsyncDrainThreshold;
+  }
+
+  /**
+   * Propagate the default options to the operation context
+   * being built up.
+   * @param roc context
+   * @return the context
+   */
+  public S3AReadOpContext applyDefaultOptions(S3AReadOpContext roc) {
+    return roc
+        .withInputPolicy(defaultInputPolicy)
+        .withChangeDetectionPolicy(changePolicy)
+        .withAsyncDrainThreshold(defaultAsyncDrainThreshold)
+        .withReadahead(defaultReadAhead);
+  }
+
+  /**
+   * Prepare to open a file from the openFile parameters.
+   * @param path path to the file
+   * @param parameters open file parameters from the builder.
+   * @param blockSize for fileStatus
+   * @return open file options
+   * @throws IOException failure to resolve the link.
+   * @throws IllegalArgumentException unknown mandatory key
+   */
+  @SuppressWarnings("ChainOfInstanceofChecks")
+  public OpenFileInformation prepareToOpenFile(
+      final Path path,
+      final OpenFileParameters parameters,
+      final long blockSize) throws IOException {
+    Configuration options = parameters.getOptions();
+    Set<String> mandatoryKeys = parameters.getMandatoryKeys();
+    String sql = options.get(SelectConstants.SELECT_SQL, null);
+    boolean isSelect = sql != null;
+    // choice of keys depends on open type
+    if (isSelect) {
+      // S3 Select call adds a large set of supported mandatory keys
+      rejectUnknownMandatoryKeys(
+          mandatoryKeys,
+          InternalSelectConstants.SELECT_OPTIONS,
+          "for " + path + " in S3 Select operation");
+    } else {
+      rejectUnknownMandatoryKeys(
+          mandatoryKeys,
+          InternalConstants.S3A_OPENFILE_KEYS,
+          "for " + path + " in non-select file I/O");
+    }
+
+    // where does a read end?
+    long fileLength = LENGTH_UNKNOWN;
+
+    // was a status passed in via a withStatus() invocation in
+    // the builder API?
+    FileStatus providedStatus = parameters.getStatus();
+    S3AFileStatus fileStatus = null;
+    if (providedStatus != null) {
+      // there's a file status
+
+      // make sure the file name matches -the rest of the path
+      // MUST NOT be checked.
+      Path providedStatusPath = providedStatus.getPath();
+      checkArgument(path.getName().equals(providedStatusPath.getName()),
+          "Filename mismatch between file being opened %s and"
+              + " supplied filestatus %s",
+          path, providedStatusPath);
+
+      // make sure the status references a file
+      if (providedStatus.isDirectory()) {
+        throw new FileNotFoundException(
+            "Supplied status references a directory " + providedStatus);
+      }
+      // build up the values
+      long len = providedStatus.getLen();
+      long modTime = providedStatus.getModificationTime();
+      String versionId;
+      String eTag;
+      // can use this status to skip our own probes,
+      LOG.debug("File was opened with a supplied FileStatus;"
+              + " skipping getFileStatus call in open() operation: {}",
+          providedStatus);
+
+      // what type is the status (and hence: what information does it contain?)
+      if (providedStatus instanceof S3AFileStatus) {
+        // is it an S3AFileSystem status?
+        S3AFileStatus st = (S3AFileStatus) providedStatus;
+        versionId = st.getVersionId();
+        eTag = st.getEtag();
+      } else if (providedStatus instanceof S3ALocatedFileStatus) {
+
+        //  S3ALocatedFileStatus instance may supply etag and version.
+        S3ALocatedFileStatus st = (S3ALocatedFileStatus) providedStatus;
+        versionId = st.getVersionId();
+        eTag = st.getEtag();
+      } else {
+        // it is another type.
+        // build a status struct without etag or version.
+        LOG.debug("Converting file status {}", providedStatus);
+        versionId = null;
+        eTag = null;
+      }
+      // Construct a new file status with the real path of the file.
+      fileStatus = new S3AFileStatus(
+          len,
+          modTime,
+          path,
+          blockSize,
+          username,
+          eTag,
+          versionId);
+      // set the end of the read to the file length
+      fileLength = fileStatus.getLen();
+    }
+    // determine start and end of file.
+    long splitStart = options.getLong(FS_OPTION_OPENFILE_SPLIT_START, 0);
+
+    // split end
+    long splitEnd = options.getLong(FS_OPTION_OPENFILE_SPLIT_END,
+        LENGTH_UNKNOWN);
+    if (splitStart > 0 && splitStart > splitEnd) {
+      LOG.warn("Split start {} is greater than split end {}, resetting",
+          splitStart, splitEnd);
+      splitStart = 0;
+    }
+
+    // read end is the open file value
+    fileLength = options.getLong(FS_OPTION_OPENFILE_LENGTH, fileLength);
+
+    // if the read end has come from options, use that
+    // in creating a file status
+    if (fileLength >= 0 && fileStatus == null) {
+      fileStatus = createStatus(path, fileLength, blockSize);
+    }
+
+    // Build up the input policy.
+    // seek policy from default, s3a opt or standard option
+    // read from the FS standard option.
+    Collection<String> policies =
+        options.getStringCollection(FS_OPTION_OPENFILE_READ_POLICY);
+    if (policies.isEmpty()) {
+      // fall back to looking at the S3A-specific option.
+      policies = options.getStringCollection(INPUT_FADVISE);
+    }
+
+    return new OpenFileInformation()
+        .withS3Select(isSelect)
+        .withSql(sql)
+        .withAsyncDrainThreshold(
+            options.getLong(ASYNC_DRAIN_THRESHOLD,
+                defaultReadAhead))
+        .withBufferSize(
+            options.getInt(FS_OPTION_OPENFILE_BUFFER_SIZE, defaultBufferSize))
+        .withChangePolicy(changePolicy)
+        .withFileLength(fileLength)
+        .withInputPolicy(
+            S3AInputPolicy.getFirstSupportedPolicy(policies, defaultInputPolicy))
+        .withReadAheadRange(
+            options.getLong(READAHEAD_RANGE, defaultReadAhead))
+        .withSplitStart(splitStart)
+        .withSplitEnd(splitEnd)
+        .withStatus(fileStatus)
+        .build();
+
+  }
+
+  /**
+   * Create a minimal file status.
+   * @param path path
+   * @param length file length/read end
+   * @param blockSize block size
+   * @return a new status
+   */
+  private S3AFileStatus createStatus(Path path, long length, long blockSize) {
+    return new S3AFileStatus(
+        length,
+        0,
+        path,
+        blockSize,
+        username,
+        null,
+        null);
+  }
+
+  /**
+   * Open a simple file, using all the default
+   * options.
+   * @return the parameters needed to open a file through
+   * {@code open(path, bufferSize)}.
+   * @param bufferSize  buffer size
+   */
+  public OpenFileInformation openSimpleFile(final int bufferSize) {
+    return new OpenFileInformation()
+        .withS3Select(false)
+        .withAsyncDrainThreshold(defaultAsyncDrainThreshold)
+        .withBufferSize(bufferSize)
+        .withChangePolicy(changePolicy)
+        .withFileLength(LENGTH_UNKNOWN)
+        .withInputPolicy(defaultInputPolicy)
+        .withReadAheadRange(defaultReadAhead)
+        .withSplitStart(0)
+        .withSplitEnd(LENGTH_UNKNOWN)
+        .build();
+  }
+
+  @Override
+  public String toString() {
+    return "OpenFileSupport{" +
+        "changePolicy=" + changePolicy +
+        ", defaultReadAhead=" + defaultReadAhead +
+        ", defaultBufferSize=" + defaultBufferSize +
+        ", defaultAsyncDrainThreshold=" + defaultAsyncDrainThreshold +
+        ", defaultInputPolicy=" + defaultInputPolicy +
+        '}';
+  }
+
+  /**
+   * The information on a file needed to open it.
+   */
+  public static final class OpenFileInformation {
+
+    /** Is this SQL? */
+    private boolean isS3Select;
+
+    /** File status; may be null. */
+    private S3AFileStatus status;
+
+    /** SQL string if this is a SQL select file. */
+    private String sql;
+
+    /** Active input policy. */
+    private S3AInputPolicy inputPolicy;
+
+    /** Change detection policy. */
+    private ChangeDetectionPolicy changePolicy;
+
+    /** Read ahead range. */
+    private long readAheadRange;
+
+    /** Buffer size. Currently ignored. */
+    private int bufferSize;
+
+    /**
+     * Where does the read start from. 0 unless known.
+     */
+    private long splitStart;
+
+    /**
+     * What is the split end?
+     * Negative if not known.
+     */
+    private long splitEnd = -1;
+
+    /**
+     * What is the file length?
+     * Negative if not known.
+     */
+    private long fileLength = -1;
+
+    /**
+     * Threshold for stream reads to switch to
+     * asynchronous draining.
+     */
+    private long asyncDrainThreshold;
+
+    /**
+     * Constructor.
+     */
+    public OpenFileInformation() {
+    }
+
+    /**
+     * Build.
+     * @return this object
+     */
+    public OpenFileInformation build() {
+      return this;
+    }
+
+    public boolean isS3Select() {
+      return isS3Select;
+    }
+
+    public S3AFileStatus getStatus() {
+      return status;
+    }
+
+    public String getSql() {
+      return sql;
+    }
+
+    public S3AInputPolicy getInputPolicy() {
+      return inputPolicy;
+    }
+
+    public ChangeDetectionPolicy getChangePolicy() {
+      return changePolicy;
+    }
+
+    public long getReadAheadRange() {
+      return readAheadRange;
+    }
+
+    public int getBufferSize() {
+      return bufferSize;
+    }
+
+    public long getSplitStart() {
+      return splitStart;
+    }
+
+    public long getSplitEnd() {
+      return splitEnd;
+    }
+
+    @Override
+    public String toString() {
+      return "OpenFileInformation{" +
+          "isSql=" + isS3Select +
+          ", status=" + status +
+          ", sql='" + sql + '\'' +
+          ", inputPolicy=" + inputPolicy +
+          ", changePolicy=" + changePolicy +
+          ", readAheadRange=" + readAheadRange +
+          ", splitStart=" + splitStart +
+          ", splitEnd=" + splitEnd +
+          ", bufferSize=" + bufferSize +
+          ", drainThreshold=" + asyncDrainThreshold +
+          '}';
+    }
+
+    /**
+     * Get the file length.
+     * @return the file length; -1 if not known.
+     */
+    public long getFileLength() {
+      return fileLength;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withS3Select(final boolean value) {
+      isS3Select = value;
+      return this;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withStatus(final S3AFileStatus value) {
+      status = value;
+      return this;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withSql(final String value) {
+      sql = value;
+      return this;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withInputPolicy(final S3AInputPolicy value) {
+      inputPolicy = value;
+      return this;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withChangePolicy(final ChangeDetectionPolicy value) {
+      changePolicy = value;
+      return this;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withReadAheadRange(final long value) {
+      readAheadRange = value;
+      return this;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withBufferSize(final int value) {
+      bufferSize = value;
+      return this;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withSplitStart(final long value) {
+      splitStart = value;
+      return this;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withSplitEnd(final long value) {
+      splitEnd = value;
+      return this;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withFileLength(final long value) {
+      fileLength = value;
+      return this;
+    }
+
+    /**
+     * Set builder value.
+     * @param value new value
+     * @return the builder
+     */
+    public OpenFileInformation withAsyncDrainThreshold(final long value) {
+      asyncDrainThreshold = value;
+      return this;
+    }
+
+    /**
+     * Propagate the options to the operation context
+     * being built up.
+     * @param roc context
+     * @return the context
+     */
+    public S3AReadOpContext applyOptions(S3AReadOpContext roc) {
+      return roc
+          .withInputPolicy(inputPolicy)
+          .withChangeDetectionPolicy(changePolicy)
+          .withAsyncDrainThreshold(asyncDrainThreshold)
+          .withReadahead(readAheadRange);
+    }
+
+  }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
index 1f8be586750..1d52b0a34ea 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FilterFileSystem;
+import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.MultipartUtils;
@@ -426,7 +427,8 @@ public abstract class S3GuardTool extends Configured implements Tool,
       String encryption =
           printOption(out, "\tEncryption", Constants.S3_ENCRYPTION_ALGORITHM,
               "none");
-      printOption(out, "\tInput seek policy", INPUT_FADVISE, INPUT_FADV_NORMAL);
+      printOption(out, "\tInput seek policy", INPUT_FADVISE,
+          Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT);
       printOption(out, "\tChange Detection Source", CHANGE_DETECT_SOURCE,
           CHANGE_DETECT_SOURCE_DEFAULT);
       printOption(out, "\tChange Detection Mode", CHANGE_DETECT_MODE,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java
index 4f387d8ccfa..fbf5226afb8 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java
@@ -71,7 +71,7 @@ public final class InternalSelectConstants {
         CSV_OUTPUT_QUOTE_FIELDS,
         CSV_OUTPUT_RECORD_DELIMITER
     ));
-    options.addAll(InternalConstants.STANDARD_OPENFILE_KEYS);
+    options.addAll(InternalConstants.S3A_OPENFILE_KEYS);
     SELECT_OPTIONS = Collections.unmodifiableSet(options);
   }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectTool.java
index 73a08750053..7a6c1afdc1f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectTool.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectTool.java
@@ -39,12 +39,12 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.impl.FutureIOSupport;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool;
 import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.util.DurationInfo;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.OperationDuration;
+import org.apache.hadoop.util.functional.FutureIO;
 
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
@@ -261,7 +261,7 @@ public class SelectTool extends S3GuardTool {
     FSDataInputStream stream;
     try(DurationInfo ignored =
             new DurationInfo(LOG, "Selecting stream")) {
-      stream = FutureIOSupport.awaitFuture(builder.build());
+      stream = FutureIO.awaitFuture(builder.build());
     } catch (FileNotFoundException e) {
       // the source file is missing.
       throw notFound(e);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java
index 328d9f7c4ce..539af2bde36 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java
@@ -188,4 +188,11 @@ public interface S3AInputStreamStatistics extends AutoCloseable,
    */
   DurationTracker initiateGetRequest();
 
+  /**
+   * Initiate a stream close/abort.
+   * @param abort was the stream aborted?
+   * @return duration tracker;
+   */
+  DurationTracker initiateInnerStreamClose(boolean abort);
+
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java
index 58bf60ec3ab..f618270798e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java
@@ -337,6 +337,10 @@ public final class EmptyS3AStatisticsContext implements S3AStatisticsContext {
       return stubDurationTracker();
     }
 
+    @Override
+    public DurationTracker initiateInnerStreamClose(final boolean abort) {
+      return stubDurationTracker();
+    }
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index df08a969e95..f4a2f036ce3 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -1017,6 +1017,14 @@ options are covered in [Testing](./testing.md).
   any call to setReadahead() is made to an open stream.</description>
 </property>
 
+<property>
+  <name>fs.s3a.input.async.drain.threshold</name>
+  <value>64K</value>
+  <description>Bytes to read ahead during a seek() before closing and
+  re-opening the S3 HTTP connection. This option will be overridden if
+  any call to setReadahead() is made to an open stream.</description>
+</property>
+
 <property>
   <name>fs.s3a.list.version</name>
   <value>2</value>
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
index 4765fa8e8d7..82a7a3c63b3 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java
@@ -18,10 +18,22 @@
 
 package org.apache.hadoop.fs.contract.s3a;
 
+import java.io.FileNotFoundException;
+
+import org.junit.Test;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
 /**
  * S3A contract tests opening files.
  */
@@ -40,4 +52,59 @@ public class ITestS3AContractOpen extends AbstractContractOpenTest {
   protected boolean areZeroByteFilesEncrypted() {
     return true;
   }
+
+  @Test
+  public void testOpenFileApplyReadBadName() throws Throwable {
+    describe("use the apply sequence to read a whole file");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    touch(fs, path);
+    FileStatus st = fs.getFileStatus(path);
+    // The final element of the path is different, so
+    // openFile must fail
+    FileStatus st2 = new FileStatus(
+        0, false,
+        st.getReplication(),
+        st.getBlockSize(),
+        st.getModificationTime(),
+        st.getAccessTime(),
+        st.getPermission(),
+        st.getOwner(),
+        st.getGroup(),
+        new Path("gopher:///localhost/something.txt"));
+    intercept(IllegalArgumentException.class, () ->
+        fs.openFile(path)
+            .withFileStatus(st2)
+            .build());
+  }
+
+  /**
+   * Pass in a directory reference and expect the openFile call
+   * to fail.
+   */
+  @Test
+  public void testOpenFileDirectory() throws Throwable {
+    describe("Change the status to a directory");
+    Path path = methodPath();
+    FileSystem fs = getFileSystem();
+    int len = 4096;
+    createFile(fs, path, true,
+        dataset(len, 0x40, 0x80));
+    FileStatus st = fs.getFileStatus(path);
+    FileStatus st2 = new FileStatus(
+        len, true,
+        st.getReplication(),
+        st.getBlockSize(),
+        st.getModificationTime(),
+        st.getAccessTime(),
+        st.getPermission(),
+        st.getOwner(),
+        st.getGroup(),
+        path);
+    intercept(FileNotFoundException.class, () ->
+        fs.openFile(path)
+            .withFileStatus(st2)
+            .build());
+  }
+
 }
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
index ba07ab24002..dd41583de3f 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java
@@ -44,11 +44,11 @@ import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
 import org.apache.hadoop.util.NativeCodeLoader;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
 import static org.apache.hadoop.util.Preconditions.checkNotNull;
 import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE;
-import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADV_NORMAL;
-import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADV_RANDOM;
-import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADV_SEQUENTIAL;
 import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE;
 import static org.apache.hadoop.fs.s3a.Constants.SSL_CHANNEL_MODE;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.FS_S3A_IMPL_DISABLE_CACHE;
@@ -87,9 +87,9 @@ public class ITestS3AContractSeek extends AbstractContractSeekTest {
   @Parameterized.Parameters
   public static Collection<Object[]> params() {
     return Arrays.asList(new Object[][]{
-        {INPUT_FADV_SEQUENTIAL, Default_JSSE},
-        {INPUT_FADV_RANDOM, OpenSSL},
-        {INPUT_FADV_NORMAL, Default_JSSE_with_GCM},
+        {FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL, Default_JSSE},
+        {FS_OPTION_OPENFILE_READ_POLICY_RANDOM, OpenSSL},
+        {FS_OPTION_OPENFILE_READ_POLICY_DEFAULT, Default_JSSE_with_GCM},
     });
   }
 
@@ -215,7 +215,8 @@ public class ITestS3AContractSeek extends AbstractContractSeekTest {
   public void testReadPolicyInFS() throws Throwable {
     describe("Verify the read policy is being consistently set");
     S3AFileSystem fs = getFileSystem();
-    assertEquals(S3AInputPolicy.getPolicy(seekPolicy), fs.getInputPolicy());
+    assertEquals(S3AInputPolicy.getPolicy(seekPolicy, S3AInputPolicy.Normal),
+        fs.getInputPolicy());
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
index 7cd60cdd4da..a80a24881fa 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
@@ -70,6 +70,10 @@ public abstract class AbstractS3AMockTest {
     // use minimum multipart size for faster triggering
     conf.setLong(Constants.MULTIPART_SIZE, MULTIPART_MIN_SIZE);
     conf.setInt(Constants.S3A_BUCKET_PROBE, 1);
+    // this is so stream draining is always blocking, allowing
+    // assertions to be safely made without worrying
+    // about any race conditions
+    conf.setInt(ASYNC_DRAIN_THRESHOLD, Integer.MAX_VALUE);
     return conf;
   }
 
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
index c2b3aab0782..61d12747c0a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -456,17 +456,6 @@ public class ITestS3AConfiguration {
         tmp1.getParent(), tmp2.getParent());
   }
 
-  @Test
-  public void testReadAheadRange() throws Exception {
-    conf = new Configuration();
-    conf.set(Constants.READAHEAD_RANGE, "300K");
-    fs = S3ATestUtils.createTestFileSystem(conf);
-    assertNotNull(fs);
-    long readAheadRange = fs.getReadAheadRange();
-    assertNotNull(readAheadRange);
-    assertEquals("Read Ahead Range Incorrect.", 300 * 1024, readAheadRange);
-  }
-
   @Test
   public void testUsernameFromUGI() throws Throwable {
     final String alice = "alice";
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index d965e6e57a2..e20de5edf06 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
 import org.apache.hadoop.util.DurationInfo;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.functional.CallableRaisingIOE;
+import org.apache.hadoop.util.functional.FutureIO;
 
 import com.amazonaws.auth.AWSCredentialsProvider;
 import org.assertj.core.api.Assertions;
@@ -91,7 +92,6 @@ import static org.apache.hadoop.util.Preconditions.checkNotNull;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CREDENTIAL_PROVIDER_PATH;
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
-import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.buildEncryptionSecrets;
@@ -1257,7 +1257,7 @@ public final class S3ATestUtils {
             .withFileStatus(status)
             .build();
 
-    try (FSDataInputStream in = awaitFuture(future)) {
+    try (FSDataInputStream in = FutureIO.awaitFuture(future)) {
       byte[] buf = new byte[(int) status.getLen()];
       in.readFully(0, buf);
       return new String(buf);
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 db5b5b56851..62f5bff35c4 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
@@ -22,6 +22,7 @@ import javax.net.ssl.SSLException;
 import java.io.IOException;
 import java.net.SocketException;
 import java.nio.charset.StandardCharsets;
+import java.util.concurrent.CompletableFuture;
 
 import com.amazonaws.SdkClientException;
 import com.amazonaws.services.s3.model.GetObjectRequest;
@@ -34,9 +35,10 @@ import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.audit.impl.NoopSpan;
 import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
-import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
 
 import static java.lang.Math.min;
+import static org.apache.hadoop.util.functional.FutureIO.eval;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 
@@ -54,7 +56,6 @@ public class TestS3AInputStreamRetry extends AbstractS3AMockTest {
   @Test
   public void testInputStreamReadRetryForException() throws IOException {
     S3AInputStream s3AInputStream = getMockedS3AInputStream();
-
     assertEquals("'a' from the test input stream 'ab' should be the first " +
         "character being read", INPUT.charAt(0), s3AInputStream.read());
     assertEquals("'b' from the test input stream 'ab' should be the second " +
@@ -103,13 +104,14 @@ public class TestS3AInputStreamRetry extends AbstractS3AMockTest {
         INPUT.length());
 
     S3AReadOpContext s3AReadOpContext = fs.createReadContext(
-        s3AFileStatus, S3AInputPolicy.Normal,
-        ChangeDetectionPolicy.getPolicy(fs.getConf()), 100, NoopSpan.INSTANCE);
+        s3AFileStatus,
+        NoopSpan.INSTANCE);
 
     return new S3AInputStream(
         s3AReadOpContext,
         s3ObjectAttributes,
-        getMockedInputStreamCallback());
+        getMockedInputStreamCallback(),
+        s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics());
   }
 
   /**
@@ -151,6 +153,11 @@ public class TestS3AInputStreamRetry extends AbstractS3AMockTest {
         return new GetObjectRequest(fs.getBucket(), key);
       }
 
+      @Override
+      public <T> CompletableFuture<T> submit(final CallableRaisingIOE<T> operation) {
+        return eval(operation);
+      }
+
       @Override
       public void close() {
       }
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java
index c858c9933fc..204f1aa0939 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java
@@ -33,6 +33,7 @@ import java.util.Date;
 import static org.junit.Assert.assertEquals;
 
 import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -59,6 +60,8 @@ public class TestS3AUnbuffer extends AbstractS3AMockTest {
     // Create mock S3ObjectInputStream and S3Object for open()
     S3ObjectInputStream objectStream = mock(S3ObjectInputStream.class);
     when(objectStream.read()).thenReturn(-1);
+    when(objectStream.read(any(byte[].class))).thenReturn(-1);
+    when(objectStream.read(any(byte[].class), anyInt(), anyInt())).thenReturn(-1);
 
     S3Object s3Object = mock(S3Object.class);
     when(s3Object.getObjectContent()).thenReturn(objectStream);
@@ -67,7 +70,7 @@ public class TestS3AUnbuffer extends AbstractS3AMockTest {
 
     // Call read and then unbuffer
     FSDataInputStream stream = fs.open(path);
-    assertEquals(0, stream.read(new byte[8])); // mocks read 0 bytes
+    assertEquals(-1, stream.read(new byte[8])); // mocks read 0 bytes
     stream.unbuffer();
 
     // Verify that unbuffer closed the object stream
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
index 4d3930fbc3c..42de7cdffc8 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java
@@ -79,7 +79,7 @@ public class TestStreamChangeTracker extends HadoopTestBase {
   public void testVersionCheckingHandlingNoVersionsVersionRequired()
       throws Throwable {
     LOG.info("If an endpoint doesn't return versions but we are configured to"
-        + "require them");
+        + " require them");
     ChangeTracker tracker = newTracker(
         ChangeDetectionPolicy.Mode.Client,
         ChangeDetectionPolicy.Source.VersionId,
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileSupport.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileSupport.java
new file mode 100644
index 00000000000..17f210dd586
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileSupport.java
@@ -0,0 +1,429 @@
+/*
+ * 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.impl;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.assertj.core.api.Assertions;
+import org.assertj.core.api.ObjectAssert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.impl.OpenFileParameters;
+import org.apache.hadoop.fs.s3a.S3AFileStatus;
+import org.apache.hadoop.fs.s3a.S3AInputPolicy;
+import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
+import org.apache.hadoop.test.HadoopTestBase;
+
+import static java.util.Collections.singleton;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START;
+import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ASYNC_DRAIN_THRESHOLD;
+import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE;
+import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Unit tests for {@link OpenFileSupport} and the associated
+ * seek policy lookup in {@link S3AInputPolicy}.
+ */
+public class TestOpenFileSupport extends HadoopTestBase {
+
+  private static final ChangeDetectionPolicy CHANGE_POLICY =
+      ChangeDetectionPolicy.createPolicy(
+          ChangeDetectionPolicy.Mode.Server,
+          ChangeDetectionPolicy.Source.None,
+          false);
+
+  private static final long READ_AHEAD_RANGE = 16;
+
+  private static final String USERNAME = "hadoop";
+
+  public static final S3AInputPolicy INPUT_POLICY = S3AInputPolicy.Sequential;
+
+  public static final String TESTFILE = "s3a://bucket/name";
+
+  private static final Path TESTPATH = new Path(TESTFILE);
+
+  /**
+   * Create a OpenFileSupport instance.
+   */
+  private static final OpenFileSupport PREPARE =
+      new OpenFileSupport(
+          CHANGE_POLICY,
+          READ_AHEAD_RANGE,
+          USERNAME,
+          IO_FILE_BUFFER_SIZE_DEFAULT,
+          DEFAULT_ASYNC_DRAIN_THRESHOLD,
+          INPUT_POLICY);
+
+  @Test
+  public void testSimpleFile() throws Throwable {
+    ObjectAssert<OpenFileSupport.OpenFileInformation>
+        asst = assertFileInfo(
+            PREPARE.openSimpleFile(1024));
+
+    asst.extracting(f -> f.getChangePolicy())
+        .isEqualTo(CHANGE_POLICY);
+    asst.extracting(f -> f.getInputPolicy())
+        .isEqualTo(INPUT_POLICY);
+    asst.extracting(f -> f.getReadAheadRange())
+        .isEqualTo(READ_AHEAD_RANGE);
+  }
+
+  /**
+   * Initiate an assert from an open file information instance.
+   * @param fi file info
+   * @return an assert stream.
+   */
+  private ObjectAssert<OpenFileSupport.OpenFileInformation> assertFileInfo(
+      final OpenFileSupport.OpenFileInformation fi) {
+    return Assertions.assertThat(fi)
+        .describedAs("File Information %s", fi);
+  }
+
+  /**
+   * Create an assertion about the openFile information from a configuration
+   * with the given key/value option.
+   * @param key key to set.
+   * @param option option value.
+   * @return the constructed OpenFileInformation.
+   */
+  public ObjectAssert<OpenFileSupport.OpenFileInformation> assertOpenFile(
+      final String key,
+      final String option) throws IOException {
+    return assertFileInfo(prepareToOpenFile(params(key, option)));
+  }
+
+  @Test
+  public void testUnknownMandatoryOption() throws Throwable {
+
+    String key = "unknown";
+    intercept(IllegalArgumentException.class, key, () ->
+        prepareToOpenFile(params(key, "undefined")));
+  }
+
+  @Test
+  public void testSeekRandomIOPolicy() throws Throwable {
+
+    // ask for random IO
+    String option = FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
+
+    // is picked up
+    assertOpenFile(INPUT_FADVISE, option)
+        .extracting(f -> f.getInputPolicy())
+        .isEqualTo(S3AInputPolicy.Random);
+    // and as neither status nor length was set: no file status
+    assertOpenFile(INPUT_FADVISE, option)
+        .extracting(f -> f.getStatus())
+        .isNull();
+  }
+
+  /**
+   * There's a standard policy name. 'adaptive',
+   * meaning 'whatever this stream does to adapt to the client's use'.
+   * On the S3A connector that is mapped to {@link S3AInputPolicy#Normal}.
+   */
+  @Test
+  public void testSeekPolicyAdaptive() throws Throwable {
+
+    // when caller asks for adaptive, they get "normal"
+    assertOpenFile(FS_OPTION_OPENFILE_READ_POLICY,
+        FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE)
+        .extracting(f -> f.getInputPolicy())
+        .isEqualTo(S3AInputPolicy.Normal);
+  }
+
+  /**
+   * Verify that an unknown seek policy falls back to
+   * {@link S3AInputPolicy#Normal}.
+   */
+  @Test
+  public void testUnknownSeekPolicyS3AOption() throws Throwable {
+    // fall back to the normal seek policy.
+    assertOpenFile(INPUT_FADVISE, "undefined")
+        .extracting(f -> f.getInputPolicy())
+        .isEqualTo(INPUT_POLICY);
+  }
+
+  /**
+   * The S3A option also supports a list of values.
+   */
+  @Test
+  public void testSeekPolicyListS3AOption() throws Throwable {
+    // fall back to the second seek policy if the first is unknown
+    assertOpenFile(INPUT_FADVISE, "hbase, random")
+        .extracting(f -> f.getInputPolicy())
+        .isEqualTo(S3AInputPolicy.Random);
+  }
+
+  /**
+   * Verify that if a list of policies is supplied in a configuration,
+   * the first recognized policy will be adopted.
+   */
+  @Test
+  public void testSeekPolicyExtractionFromList() throws Throwable {
+    String plist = "a, b, RandOm, other ";
+    Configuration conf = conf(FS_OPTION_OPENFILE_READ_POLICY, plist);
+    Collection<String> options = conf.getTrimmedStringCollection(
+        FS_OPTION_OPENFILE_READ_POLICY);
+    Assertions.assertThat(S3AInputPolicy.getFirstSupportedPolicy(options, null))
+        .describedAs("Policy from " + plist)
+        .isEqualTo(S3AInputPolicy.Random);
+  }
+
+  @Test
+  public void testAdaptiveSeekPolicyRecognized() throws Throwable {
+    Assertions.assertThat(S3AInputPolicy.getPolicy("adaptive", null))
+        .describedAs("adaptive")
+        .isEqualTo(S3AInputPolicy.Normal);
+  }
+
+  @Test
+  public void testUnknownSeekPolicyFallback() throws Throwable {
+    Assertions.assertThat(S3AInputPolicy.getPolicy("unknown", null))
+        .describedAs("unknown policy")
+        .isNull();
+  }
+
+  /**
+   * Test the mapping of the standard option names.
+   */
+  @Test
+  public void testInputPolicyMapping() throws Throwable {
+    Object[][] policyMapping = {
+        {"normal", S3AInputPolicy.Normal},
+        {FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE, S3AInputPolicy.Normal},
+        {FS_OPTION_OPENFILE_READ_POLICY_DEFAULT, S3AInputPolicy.Normal},
+        {FS_OPTION_OPENFILE_READ_POLICY_RANDOM, S3AInputPolicy.Random},
+        {FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL, S3AInputPolicy.Sequential},
+    };
+    for (Object[] mapping : policyMapping) {
+      String name = (String) mapping[0];
+      Assertions.assertThat(S3AInputPolicy.getPolicy(name, null))
+          .describedAs("Policy %s", name)
+          .isEqualTo(mapping[1]);
+    }
+  }
+
+  /**
+   * Verify readahead range is picked up.
+   */
+  @Test
+  public void testReadahead() throws Throwable {
+    // readahead range option
+    assertOpenFile(READAHEAD_RANGE, "4096")
+        .extracting(f -> f.getReadAheadRange())
+        .isEqualTo(4096L);
+  }
+
+  /**
+   * Verify buffer size  is picked up.
+   */
+  @Test
+  public void testBufferSize() throws Throwable {
+    // readahead range option
+    assertOpenFile(FS_OPTION_OPENFILE_BUFFER_SIZE, "4096")
+        .extracting(f -> f.getBufferSize())
+        .isEqualTo(4096);
+  }
+
+  @Test
+  public void testStatusWithValidFilename() throws Throwable {
+    Path p = new Path("file:///tmp/" + TESTPATH.getName());
+    ObjectAssert<OpenFileSupport.OpenFileInformation> asst =
+        assertFileInfo(prepareToOpenFile(
+            params(FS_OPTION_OPENFILE_LENGTH, "32")
+                .withStatus(status(p, 4096))));
+    asst.extracting(f -> f.getStatus().getVersionId())
+        .isEqualTo("version");
+    asst.extracting(f -> f.getStatus().getEtag())
+        .isEqualTo("etag");
+    asst.extracting(f -> f.getStatus().getLen())
+        .isEqualTo(4096L);
+  }
+
+  /**
+   * Verify S3ALocatedFileStatus is handled.
+   */
+  @Test
+  public void testLocatedStatus() throws Throwable {
+    Path p = new Path("file:///tmp/" + TESTPATH.getName());
+    ObjectAssert<OpenFileSupport.OpenFileInformation> asst =
+        assertFileInfo(
+            prepareToOpenFile(
+                params(FS_OPTION_OPENFILE_LENGTH, "32")
+                    .withStatus(
+                        new S3ALocatedFileStatus(
+                            status(p, 4096), null))));
+    asst.extracting(f -> f.getStatus().getVersionId())
+        .isEqualTo("version");
+    asst.extracting(f -> f.getStatus().getEtag())
+        .isEqualTo("etag");
+    asst.extracting(f -> f.getStatus().getLen())
+        .isEqualTo(4096L);
+  }
+
+  /**
+   * Callers cannot supply a directory status when opening a file.
+   */
+  @Test
+  public void testDirectoryStatus() throws Throwable {
+    intercept(FileNotFoundException.class, TESTFILE, () ->
+        prepareToOpenFile(
+            params(INPUT_FADVISE, "normal")
+                .withStatus(new S3AFileStatus(true, TESTPATH, USERNAME))));
+  }
+
+  /**
+   * File name must match the path argument to openFile().
+   */
+  @Test
+  public void testStatusWithInconsistentFilename() throws Throwable {
+    intercept(IllegalArgumentException.class, TESTFILE, () ->
+        prepareToOpenFile(params(INPUT_FADVISE, "normal")
+            .withStatus(new S3AFileStatus(true,
+                new Path(TESTFILE + "-"), USERNAME))));
+  }
+
+  /**
+   * Prepare to open a file with the set of parameters.
+   * @param parameters open a file
+   * @return
+   * @throws IOException
+   */
+  public OpenFileSupport.OpenFileInformation prepareToOpenFile(
+      final OpenFileParameters parameters)
+      throws IOException {
+    return PREPARE.prepareToOpenFile(TESTPATH,
+        parameters,
+        IO_FILE_BUFFER_SIZE_DEFAULT
+    );
+  }
+
+  /**
+   * If a file length option is set, a file status
+   * is created.
+   */
+  @Test
+  public void testFileLength() throws Throwable {
+    ObjectAssert<OpenFileSupport.OpenFileInformation> asst =
+        assertFileInfo(prepareToOpenFile(
+            params(FS_OPTION_OPENFILE_LENGTH, "8192")
+                .withStatus(null)));
+    asst.extracting(f -> f.getStatus())
+        .isNotNull();
+    asst.extracting(f -> f.getStatus().getPath())
+        .isEqualTo(TESTPATH);
+    asst.extracting(f -> f.getStatus().getLen())
+        .isEqualTo(8192L);
+  }
+
+  /**
+   * Verify that setting the split end sets the length.
+   * By passing in a value greater than the size of an int,
+   * the test verifies that the long is passed everywhere.
+   */
+  @Test
+  public void testSplitEndSetsLength() throws Throwable {
+    long bigFile = 2L ^ 34;
+    assertOpenFile(FS_OPTION_OPENFILE_SPLIT_END, Long.toString(bigFile))
+        .matches(p -> p.getSplitEnd() == bigFile, "split end")
+        .matches(p -> p.getFileLength() == -1, "file length")
+        .matches(p -> p.getStatus() == null, "status");
+  }
+
+  /**
+   * Semantics of split and length. Split end can only be safely treated
+   * as a hint unless the codec is known (how?) that it will never
+   * read past it.
+   */
+  @Test
+  public void testSplitEndAndLength() throws Throwable {
+    long splitEnd = 256;
+    long len = 8192;
+    Configuration conf = conf(FS_OPTION_OPENFILE_LENGTH,
+        Long.toString(len));
+    conf.setLong(FS_OPTION_OPENFILE_SPLIT_END, splitEnd);
+    conf.setLong(FS_OPTION_OPENFILE_SPLIT_START, 1024);
+    Set<String> s = new HashSet<>();
+    Collections.addAll(s,
+        FS_OPTION_OPENFILE_SPLIT_START,
+        FS_OPTION_OPENFILE_SPLIT_END,
+        FS_OPTION_OPENFILE_LENGTH);
+    assertFileInfo(prepareToOpenFile(
+        new OpenFileParameters()
+            .withMandatoryKeys(s)
+            .withOptions(conf)))
+        .matches(p -> p.getSplitStart() == 0, "split start")
+        .matches(p -> p.getSplitEnd() == splitEnd, "split end")
+        .matches(p -> p.getStatus().getLen() == len, "file length");
+  }
+
+  /**
+   * Create an S3A status entry with stub etag and versions, timestamp of 0.
+   * @param path status path
+   * @param length file length
+   * @return a status instance.
+   */
+  private S3AFileStatus status(final Path path, final int length) {
+    return new S3AFileStatus(length, 0,
+        path, 0, "", "etag", "version");
+  }
+
+  /**
+   * Create an instance of {@link OpenFileParameters} with
+   * the key as a mandatory parameter.
+   * @param key mandatory key
+   * @param val value
+   * @return the instance.
+   */
+  private OpenFileParameters params(final String key, final String val) {
+    return new OpenFileParameters()
+        .withMandatoryKeys(singleton(key))
+        .withOptions(conf(key, val));
+  }
+
+  /**
+   * Create a configuration with a single entry.
+   * @param key entry key
+   * @param val entry value
+   * @return a configuration
+   */
+  private Configuration conf(String key, Object val) {
+    Configuration c = new Configuration(false);
+    c.set(key, val.toString());
+    return c;
+  }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java
new file mode 100644
index 00000000000..b0ee531112b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java
@@ -0,0 +1,209 @@
+/*
+ * 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.performance;
+
+
+import java.io.EOFException;
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.readStream;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
+import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE;
+import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_OPENED;
+import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_SEEK_BYTES_SKIPPED;
+import static org.apache.hadoop.fs.s3a.performance.OperationCost.NO_HEAD_OR_LIST;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertDurationRange;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics;
+import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_FILE_OPENED;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Cost of openFile().
+ */
+public class ITestS3AOpenCost extends AbstractS3ACostTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestS3AOpenCost.class);
+
+  private Path testFile;
+
+  private FileStatus testFileStatus;
+
+  private long fileLength;
+
+  public ITestS3AOpenCost() {
+    super(true);
+  }
+
+  /**
+   * Setup creates a test file, saves is status and length
+   * to fields.
+   */
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    S3AFileSystem fs = getFileSystem();
+    testFile = methodPath();
+
+    writeTextFile(fs, testFile, "openfile", true);
+    testFileStatus = fs.getFileStatus(testFile);
+    fileLength = testFileStatus.getLen();
+  }
+
+  /**
+   * Test when openFile() performs GET requests when file status
+   * and length options are passed down.
+   * Note that the input streams only update the FS statistics
+   * in close(), so metrics cannot be verified until all operations
+   * on a stream are complete.
+   * This is slightly less than ideal.
+   */
+  @Test
+  public void testOpenFileWithStatusOfOtherFS() throws Throwable {
+    describe("Test cost of openFile with/without status; raw only");
+    S3AFileSystem fs = getFileSystem();
+
+    // now read that file back in using the openFile call.
+    // with a new FileStatus and a different path.
+    // this verifies that any FileStatus class/subclass is used
+    // as a source of the file length.
+    FileStatus st2 = new FileStatus(
+        fileLength, false,
+        testFileStatus.getReplication(),
+        testFileStatus.getBlockSize(),
+        testFileStatus.getModificationTime(),
+        testFileStatus.getAccessTime(),
+        testFileStatus.getPermission(),
+        testFileStatus.getOwner(),
+        testFileStatus.getGroup(),
+        new Path("gopher:///localhost/" + testFile.getName()));
+
+    // no IO in open
+    FSDataInputStream in = verifyMetrics(() ->
+            fs.openFile(testFile)
+                .withFileStatus(st2)
+                .build()
+                .get(),
+        always(NO_HEAD_OR_LIST),
+        with(STREAM_READ_OPENED, 0));
+
+    // the stream gets opened during read
+    long readLen = verifyMetrics(() ->
+            readStream(in),
+        always(NO_HEAD_OR_LIST),
+        with(STREAM_READ_OPENED, 1));
+    assertEquals("bytes read from file", fileLength, readLen);
+  }
+
+  @Test
+  public void testOpenFileShorterLength() throws Throwable {
+    // do a second read with the length declared as short.
+    // we now expect the bytes read to be shorter.
+    S3AFileSystem fs = getFileSystem();
+
+    S3ATestUtils.MetricDiff bytesDiscarded =
+        new S3ATestUtils.MetricDiff(fs, STREAM_READ_BYTES_READ_CLOSE);
+    int offset = 2;
+    long shortLen = fileLength - offset;
+    // open the file
+    FSDataInputStream in2 = verifyMetrics(() ->
+            fs.openFile(testFile)
+                .must(FS_OPTION_OPENFILE_READ_POLICY,
+                    FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL)
+                .opt(FS_OPTION_OPENFILE_LENGTH, shortLen)
+                .build()
+                .get(),
+        always(NO_HEAD_OR_LIST),
+        with(STREAM_READ_OPENED, 0));
+
+    // verify that the statistics are in range
+    IOStatistics ioStatistics = extractStatistics(in2);
+    Object statsString = demandStringifyIOStatistics(ioStatistics);
+    LOG.info("Statistics of open stream {}", statsString);
+    verifyStatisticCounterValue(ioStatistics, ACTION_FILE_OPENED, 1);
+    // no network IO happened, duration is 0. There's a very small
+    // risk of some delay making it positive just from scheduling delays
+    assertDurationRange(ioStatistics, ACTION_FILE_OPENED, 0, 0);
+    // now read it
+    long r2 = verifyMetrics(() ->
+            readStream(in2),
+        always(NO_HEAD_OR_LIST),
+        with(STREAM_READ_OPENED, 1),
+        with(STREAM_READ_BYTES_READ_CLOSE, 0),
+        with(STREAM_READ_SEEK_BYTES_SKIPPED, 0));
+
+    LOG.info("Statistics of read stream {}", statsString);
+
+    assertEquals("bytes read from file", shortLen, r2);
+    // no bytes were discarded.
+    bytesDiscarded.assertDiffEquals(0);
+  }
+
+  @Test
+  public void testOpenFileLongerLength() throws Throwable {
+    // do a second read with the length declared as longer
+    // than it is.
+    // An EOF will be read on readFully(), -1 on a read()
+
+    S3AFileSystem fs = getFileSystem();
+    // set a length past the actual file length
+    long longLen = fileLength + 10;
+    FSDataInputStream in3 = verifyMetrics(() ->
+            fs.openFile(testFile)
+                .must(FS_OPTION_OPENFILE_READ_POLICY,
+                    FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL)
+                .must(FS_OPTION_OPENFILE_LENGTH, longLen)
+                .build()
+                .get(),
+        always(NO_HEAD_OR_LIST));
+
+    // assert behaviors of seeking/reading past the file length.
+    // there is no attempt at recovery.
+    verifyMetrics(() -> {
+      byte[] out = new byte[(int) longLen];
+      intercept(EOFException.class,
+          () -> in3.readFully(0, out));
+      in3.seek(longLen - 1);
+      assertEquals("read past real EOF on " + in3,
+          -1, in3.read());
+      in3.close();
+      return in3.toString();
+    },
+        // two GET calls were made, one for readFully,
+        // the second on the read() past the EOF
+        // the operation has got as far as S3
+        with(STREAM_READ_OPENED, 2));
+
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java
index 7ae60a8c7d8..03bc10f86cd 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java
@@ -62,6 +62,12 @@ public final class OperationCost {
   public static final OperationCost NO_IO =
       new OperationCost(0, 0);
 
+  /**
+   * More detailed description of the NO_IO cost.
+   */
+  public static final OperationCost NO_HEAD_OR_LIST =
+      NO_IO;
+
   /** A HEAD operation. */
   public static final OperationCost HEAD_OPERATION = new OperationCost(1, 0);
 
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
index 27798e21ed2..d73a938bcce 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
@@ -41,7 +42,6 @@ import org.apache.hadoop.util.LineReader;
 import org.assertj.core.api.Assertions;
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -50,6 +50,9 @@ import org.slf4j.LoggerFactory;
 import java.io.EOFException;
 import java.io.IOException;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
@@ -57,20 +60,22 @@ import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatSt
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMaximumStatistic;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMeanStatistic;
 import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
-import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
 import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToString;
 import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN;
 import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /**
- * Look at the performance of S3a operations.
+ * Look at the performance of S3a Input Stream Reads.
  */
 public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
   private static final Logger LOG = LoggerFactory.getLogger(
       ITestS3AInputStreamPerformance.class);
+  private static final int READAHEAD_128K = 128 * _1KB;
 
   private S3AFileSystem s3aFS;
   private Path testData;
@@ -128,14 +133,16 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
     describe("cleanup");
     IOUtils.closeStream(in);
     if (in != null) {
+      final IOStatistics stats = in.getIOStatistics();
       LOG.info("Stream statistics {}",
-          ioStatisticsSourceToString(in));
-      IOSTATS.aggregate(in.getIOStatistics());
+          ioStatisticsToPrettyString(stats));
+      IOSTATS.aggregate(stats);
     }
     if (s3aFS != null) {
+      final IOStatistics stats = s3aFS.getIOStatistics();
       LOG.info("FileSystem statistics {}",
-          ioStatisticsSourceToString(s3aFS));
-      FILESYSTEM_IOSTATS.aggregate(s3aFS.getIOStatistics());
+          ioStatisticsToPrettyString(stats));
+      FILESYSTEM_IOSTATS.aggregate(stats);
       IOUtils.closeStream(s3aFS);
     }
   }
@@ -177,7 +184,7 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
   FSDataInputStream openTestFile(S3AInputPolicy inputPolicy, long readahead)
       throws IOException {
     requireCSVTestData();
-    return openDataFile(s3aFS, this.testData, inputPolicy, readahead);
+    return openDataFile(s3aFS, testData, inputPolicy, readahead, testDataStatus.getLen());
   }
 
   /**
@@ -187,27 +194,28 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
    * @param path path to open
    * @param inputPolicy input policy to use
    * @param readahead readahead/buffer size
+   * @param length
    * @return the stream, wrapping an S3a one
    * @throws IOException IO problems
    */
   private FSDataInputStream openDataFile(S3AFileSystem fs,
       Path path,
       S3AInputPolicy inputPolicy,
-      long readahead) throws IOException {
+      long readahead,
+      final long length) throws IOException {
     int bufferSize = getConf().getInt(KEY_READ_BUFFER_SIZE,
         DEFAULT_READ_BUFFER_SIZE);
-    S3AInputPolicy policy = fs.getInputPolicy();
-    fs.setInputPolicy(inputPolicy);
-    try {
-      FSDataInputStream stream = fs.open(path, bufferSize);
-      if (readahead >= 0) {
-        stream.setReadahead(readahead);
-      }
-      streamStatistics = getInputStreamStatistics(stream);
-      return stream;
-    } finally {
-      fs.setInputPolicy(policy);
+    final FutureDataInputStreamBuilder builder = fs.openFile(path)
+        .opt(FS_OPTION_OPENFILE_READ_POLICY,
+            inputPolicy.toString())
+        .opt(FS_OPTION_OPENFILE_LENGTH, length)
+        .opt(FS_OPTION_OPENFILE_BUFFER_SIZE, bufferSize);
+    if (readahead > 0) {
+      builder.opt(READAHEAD_RANGE, readahead);
     }
+    FSDataInputStream stream = awaitFuture(builder.build());
+    streamStatistics = getInputStreamStatistics(stream);
+    return stream;
   }
 
   /**
@@ -293,8 +301,10 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
       if (bandwidth(blockTimer, blockSize) < minimumBandwidth) {
         LOG.warn("Bandwidth {} too low on block {}: resetting connection",
             bw, blockId);
-        Assert.assertTrue("Bandwidth of " + bw +" too low after  "
-            + resetCount + " attempts", resetCount <= maxResetCount);
+        Assertions.assertThat(resetCount)
+            .describedAs("Bandwidth of %s too low after  %s attempts",
+                bw, resetCount)
+            .isLessThanOrEqualTo(maxResetCount);
         resetCount++;
         // reset the connection
         getS3AInputStream(in).resetConnection();
@@ -359,7 +369,7 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
   public void testDecompressionSequential128K() throws Throwable {
     describe("Decompress with a 128K readahead");
     skipIfClientSideEncryption();
-    executeDecompression(128 * _1KB, S3AInputPolicy.Sequential);
+    executeDecompression(READAHEAD_128K, S3AInputPolicy.Sequential);
     assertStreamOpenedExactlyOnce();
   }
 
@@ -558,7 +568,7 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
     byte[] buffer = new byte[datasetLen];
     int readahead = _8K;
     int halfReadahead = _4K;
-    in = openDataFile(fs, dataFile, S3AInputPolicy.Random, readahead);
+    in = openDataFile(fs, dataFile, S3AInputPolicy.Random, readahead, datasetLen);
 
     LOG.info("Starting initial reads");
     S3AInputStream s3aStream = getS3aStream();
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java
index 56d99d1abe3..bf5d96e73b3 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java
@@ -60,11 +60,11 @@ import org.apache.hadoop.mapreduce.lib.input.LineRecordReader;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.apache.hadoop.util.DurationInfo;
 
-import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVPath;
 import static org.apache.hadoop.fs.s3a.select.CsvFile.ALL_QUOTES;
 import static org.apache.hadoop.fs.s3a.select.SelectConstants.*;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /**
  * Superclass for S3 Select tests.
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java
index 4c173ab07b3..97fbaae0ae4 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java
@@ -63,8 +63,8 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.hadoop.util.DurationInfo;
 
-import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE;
-import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADV_NORMAL;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT;
 import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE;
 import static org.apache.hadoop.fs.s3a.select.CsvFile.ALL_QUOTES;
 import static org.apache.hadoop.fs.s3a.select.SelectBinding.expandBackslashChars;
@@ -767,7 +767,8 @@ public class ITestS3Select extends AbstractS3SelectTest {
     JobConf conf = createJobConf();
     inputOpt(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_NONE);
     inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_IGNORE);
-    inputMust(conf, INPUT_FADVISE, INPUT_FADV_NORMAL);
+    inputMust(conf, FS_OPTION_OPENFILE_READ_POLICY,
+        FS_OPTION_OPENFILE_READ_POLICY_DEFAULT);
     inputMust(conf, SELECT_ERRORS_INCLUDE_SQL, "true");
     verifySelectionCount(EVEN_ROWS_COUNT,
         SELECT_EVEN_ROWS_NO_HEADER,
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java
index 52a59138477..c1c7b89dce8 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java
@@ -30,7 +30,6 @@ import org.junit.Test;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.examples.WordCount;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.impl.FutureIOSupport;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.fs.s3a.S3AUtils;
@@ -44,6 +43,7 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.DurationInfo;
+import org.apache.hadoop.util.functional.FutureIO;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 
@@ -203,7 +203,7 @@ public class ITestS3SelectMRJob extends AbstractS3SelectTest {
   private String readStringFromFile(Path path) throws IOException {
     int bytesLen = (int)fs.getFileStatus(path).getLen();
     byte[] buffer = new byte[bytesLen];
-    return FutureIOSupport.awaitFuture(
+    return FutureIO.awaitFuture(
         fs.openFile(path).build().thenApply(in -> {
           try {
             IOUtils.readFully(in, buffer, 0, bytesLen);


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


[hadoop] 02/04: HADOOP-16202. Enhanced openFile(): mapreduce and YARN changes. (#2584/2)

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

stevel pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 6999acf520080def399b6d4b635a8ba0d0bcce94
Author: Steve Loughran <st...@cloudera.com>
AuthorDate: Sun Apr 24 17:10:34 2022 +0100

    HADOOP-16202. Enhanced openFile(): mapreduce and YARN changes. (#2584/2)
    
    These changes ensure that sequential files are opened with the
    right read policy, and split start/end is passed in.
    
    As well as offering opportunities for filesystem clients to
    choose fetch/cache/seek policies, the settings ensure that
    processing text files on an s3 bucket where the default policy
    is "random" will still be processed efficiently.
    
    This commit depends on the associated hadoop-common patch,
    which must be committed first.
    
    Contributed by Steve Loughran.
    
    Change-Id: Ic6713fd752441cf42ebe8739d05c2293a5db9f94
---
 .../jobhistory/JobHistoryCopyService.java          | 10 ++++++++-
 .../org/apache/hadoop/mapred/LineRecordReader.java | 13 +++++++++---
 .../lib/input/FixedLengthRecordReader.java         |  7 ++++---
 .../mapreduce/lib/input/LineRecordReader.java      | 14 ++++++++++---
 .../mapreduce/lib/input/NLineInputFormat.java      |  6 +++---
 .../hadoop/examples/terasort/TeraInputFormat.java  | 16 +++++++++++++--
 .../tools/mapred/RetriableFileCopyCommand.java     |  9 +++++++-
 .../streaming/mapreduce/StreamInputFormat.java     |  6 +++---
 .../yarn/logaggregation/AggregatedLogFormat.java   | 17 +++++++++++++--
 .../org/apache/hadoop/yarn/util/FSDownload.java    | 24 +++++++++++++++-------
 10 files changed, 94 insertions(+), 28 deletions(-)

diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java
index ee4ec2c86a1..ecae4f2fc06 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java
@@ -35,6 +35,10 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
+
 /**
  * Reads in history events from the JobHistoryFile and sends them out again
  * to be recorded.
@@ -118,7 +122,11 @@ public class JobHistoryCopyService extends CompositeService implements HistoryEv
         fc.makeQualified(JobHistoryUtils.getStagingJobHistoryFile(histDirPath,
           jobId, (applicationAttemptId.getAttemptId() - 1)));
     LOG.info("History file is at " + historyFile);
-    in = fc.open(historyFile);
+    in = awaitFuture(
+        fc.openFile(historyFile)
+            .opt(FS_OPTION_OPENFILE_READ_POLICY,
+                FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
+            .build());
     return in;
   }
   
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java
index 1fcb118a100..5724e729310 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Seekable;
-import org.apache.hadoop.fs.impl.FutureIOSupport;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.CodecPool;
@@ -41,9 +40,13 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.lib.input.CompressedSplitLineReader;
 import org.apache.hadoop.mapreduce.lib.input.SplitLineReader;
 import org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader;
+import org.apache.hadoop.util.functional.FutureIO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START;
+
 /**
  * Treats keys as offset in file and value as line. 
  */
@@ -109,10 +112,14 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
     // open the file and seek to the start of the split
     final FutureDataInputStreamBuilder builder =
         file.getFileSystem(job).openFile(file);
-    FutureIOSupport.propagateOptions(builder, job,
+    // the start and end of the split may be used to build
+    // an input strategy.
+    builder.opt(FS_OPTION_OPENFILE_SPLIT_START, start)
+        .opt(FS_OPTION_OPENFILE_SPLIT_END, end);
+    FutureIO.propagateOptions(builder, job,
         MRJobConfig.INPUT_FILE_OPTION_PREFIX,
         MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
-    fileIn = FutureIOSupport.awaitFuture(builder.build());
+    fileIn = FutureIO.awaitFuture(builder.build());
     if (isCompressedInput()) {
       decompressor = CodecPool.getDecompressor(codec);
       if (codec instanceof SplittableCompressionCodec) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FixedLengthRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FixedLengthRecordReader.java
index c0ae9a5cdac..6969f61836f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FixedLengthRecordReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FixedLengthRecordReader.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Seekable;
-import org.apache.hadoop.fs.impl.FutureIOSupport;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.compress.CodecPool;
@@ -40,6 +39,8 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.functional.FutureIO;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -94,10 +95,10 @@ public class FixedLengthRecordReader
     // open the file
     final FutureDataInputStreamBuilder builder =
         file.getFileSystem(job).openFile(file);
-    FutureIOSupport.propagateOptions(builder, job,
+    FutureIO.propagateOptions(builder, job,
         MRJobConfig.INPUT_FILE_OPTION_PREFIX,
         MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
-    fileIn = FutureIOSupport.awaitFuture(builder.build());
+    fileIn = FutureIO.awaitFuture(builder.build());
 
     CompressionCodec codec = new CompressionCodecFactory(job).getCodec(file);
     if (null != codec) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
index 160c7635658..617abaacae0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Seekable;
-import org.apache.hadoop.fs.impl.FutureIOSupport;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.CodecPool;
@@ -40,9 +39,14 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.functional.FutureIO;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START;
+
 /**
  * Treats keys as offset in file and value as line. 
  */
@@ -86,10 +90,14 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
     // open the file and seek to the start of the split
     final FutureDataInputStreamBuilder builder =
         file.getFileSystem(job).openFile(file);
-    FutureIOSupport.propagateOptions(builder, job,
+    // the start and end of the split may be used to build
+    // an input strategy.
+    builder.opt(FS_OPTION_OPENFILE_SPLIT_START, start);
+    builder.opt(FS_OPTION_OPENFILE_SPLIT_END, end);
+    FutureIO.propagateOptions(builder, job,
         MRJobConfig.INPUT_FILE_OPTION_PREFIX,
         MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
-    fileIn = FutureIOSupport.awaitFuture(builder.build());
+    fileIn = FutureIO.awaitFuture(builder.build());
     
     CompressionCodec codec = new CompressionCodecFactory(job).getCodec(file);
     if (null!=codec) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/NLineInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/NLineInputFormat.java
index dfff9ad0d2b..5161a96c345 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/NLineInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/NLineInputFormat.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.impl.FutureIOSupport;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -39,6 +38,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.LineReader;
+import org.apache.hadoop.util.functional.FutureIO;
 
 /**
  * NLineInputFormat which splits N lines of input as one split.
@@ -99,10 +99,10 @@ public class NLineInputFormat extends FileInputFormat<LongWritable, Text> {
     try {
       final FutureDataInputStreamBuilder builder =
           fileName.getFileSystem(conf).openFile(fileName);
-      FutureIOSupport.propagateOptions(builder, conf,
+      FutureIO.propagateOptions(builder, conf,
           MRJobConfig.INPUT_FILE_OPTION_PREFIX,
           MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
-      FSDataInputStream in  = FutureIOSupport.awaitFuture(builder.build());
+      FSDataInputStream in  = FutureIO.awaitFuture(builder.build());
       lr = new LineReader(in, conf);
       Text line = new Text();
       int numLines = 0;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java
index 20ce8ef2b60..f284a9c3807 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java
@@ -27,6 +27,7 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -41,6 +42,12 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.apache.hadoop.util.IndexedSortable;
 import org.apache.hadoop.util.QuickSort;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.functional.FutureIO;
+
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START;
 
 /**
  * An input format that reads the first 10 characters of each line as the key
@@ -224,12 +231,17 @@ public class TeraInputFormat extends FileInputFormat<Text,Text> {
         throws IOException, InterruptedException {
       Path p = ((FileSplit)split).getPath();
       FileSystem fs = p.getFileSystem(context.getConfiguration());
-      in = fs.open(p);
       long start = ((FileSplit)split).getStart();
       // find the offset to start at a record boundary
       offset = (RECORD_LENGTH - (start % RECORD_LENGTH)) % RECORD_LENGTH;
-      in.seek(start + offset);
       length = ((FileSplit)split).getLength();
+      final FutureDataInputStreamBuilder builder = fs.openFile(p)
+          .opt(FS_OPTION_OPENFILE_SPLIT_START, start)
+          .opt(FS_OPTION_OPENFILE_SPLIT_END, start + length)
+          .opt(FS_OPTION_OPENFILE_READ_POLICY,
+              FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL);
+      in = FutureIO.awaitFuture(builder.build());
+      in.seek(start + offset);
     }
 
     public void close() throws IOException {
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
index 544dbfbc4c0..ae6b734f486 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
@@ -52,7 +52,10 @@ import org.apache.hadoop.tools.util.ThrottledInputStream;
 
 import org.apache.hadoop.classification.VisibleForTesting;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
 import static org.apache.hadoop.tools.mapred.CopyMapper.getFileAttributeSettings;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /**
  * This class extends RetriableCommand to implement the copy of files,
@@ -362,7 +365,11 @@ public class RetriableFileCopyCommand extends RetriableCommand {
       FileSystem fs = path.getFileSystem(conf);
       float bandwidthMB = conf.getFloat(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
               DistCpConstants.DEFAULT_BANDWIDTH_MB);
-      FSDataInputStream in = fs.open(path);
+      // open with sequential read, but not whole-file
+      FSDataInputStream in = awaitFuture(fs.openFile(path)
+          .opt(FS_OPTION_OPENFILE_READ_POLICY,
+              FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL)
+          .build());
       return new ThrottledInputStream(in, bandwidthMB * 1024 * 1024);
     }
     catch (IOException e) {
diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamInputFormat.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamInputFormat.java
index 77f4e041d5f..f44488c7c02 100644
--- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamInputFormat.java
+++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamInputFormat.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.impl.FutureIOSupport;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -35,6 +34,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.hadoop.mapreduce.lib.input.KeyValueTextInputFormat;
 import org.apache.hadoop.streaming.StreamUtil;
+import org.apache.hadoop.util.functional.FutureIO;
 
 /**
  * An input format that selects a RecordReader based on a JobConf property. This
@@ -66,10 +66,10 @@ public class StreamInputFormat extends KeyValueTextInputFormat {
     FileSystem fs = path.getFileSystem(conf);
     // open the file
     final FutureDataInputStreamBuilder builder = fs.openFile(path);
-    FutureIOSupport.propagateOptions(builder, conf,
+    FutureIO.propagateOptions(builder, conf,
         MRJobConfig.INPUT_FILE_OPTION_PREFIX,
         MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
-    FSDataInputStream in = FutureIOSupport.awaitFuture(builder.build());
+    FSDataInputStream in = FutureIO.awaitFuture(builder.build());
 
     // Factory dispatch based on available params..
     Class readerClass;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index 3f251c74bea..5a49f9ff501 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -77,6 +78,11 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables;
 
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
+
 @Public
 @Evolving
 public class AggregatedLogFormat {
@@ -576,9 +582,16 @@ public class AggregatedLogFormat {
       try {
         FileContext fileContext =
             FileContext.getFileContext(remoteAppLogFile.toUri(), conf);
-        this.fsDataIStream = fileContext.open(remoteAppLogFile);
+        FileStatus status = fileContext.getFileStatus(remoteAppLogFile);
+        this.fsDataIStream = awaitFuture(
+            fileContext.openFile(remoteAppLogFile)
+                .opt(FS_OPTION_OPENFILE_READ_POLICY,
+                    FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL)
+                .opt(FS_OPTION_OPENFILE_LENGTH,
+                    status.getLen())   // file length hint for object stores
+                .build());
         reader = new TFile.Reader(this.fsDataIStream,
-            fileContext.getFileStatus(remoteAppLogFile).getLen(), conf);
+            status.getLen(), conf);
         this.scanner = reader.createScanner();
       } catch (IOException ioe) {
         close();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
index e24b96cd32f..56808c75ff6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java
@@ -60,7 +60,11 @@ import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
-/**
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
+
+ /**
  * Download a single URL to the local disk.
  *
  */
@@ -285,23 +289,25 @@ public class FSDownload implements Callable<Path> {
       }
     }
 
-    downloadAndUnpack(sCopy, destination);
+    downloadAndUnpack(sCopy, sStat, destination);
   }
 
   /**
    * Copy source path to destination with localization rules.
-   * @param source source path to copy. Typically HDFS
+   * @param source source path to copy. Typically HDFS or an object store.
+   * @param sourceStatus status of source
    * @param destination destination path. Typically local filesystem
    * @exception YarnException Any error has occurred
    */
-  private void downloadAndUnpack(Path source, Path destination)
+  private void downloadAndUnpack(Path source,
+      FileStatus sourceStatus,  Path destination)
       throws YarnException {
     try {
       FileSystem sourceFileSystem = source.getFileSystem(conf);
       FileSystem destinationFileSystem = destination.getFileSystem(conf);
-      if (sourceFileSystem.getFileStatus(source).isDirectory()) {
+      if (sourceStatus.isDirectory()) {
         FileUtil.copy(
-            sourceFileSystem, source,
+            sourceFileSystem, sourceStatus,
             destinationFileSystem, destination, false,
             true, conf);
       } else {
@@ -329,7 +335,11 @@ public class FSDownload implements Callable<Path> {
                       FileSystem sourceFileSystem,
                       FileSystem destinationFileSystem)
       throws IOException, InterruptedException, ExecutionException {
-    try (InputStream inputStream = sourceFileSystem.open(source)) {
+    try (InputStream inputStream = awaitFuture(
+        sourceFileSystem.openFile(source)
+            .opt(FS_OPTION_OPENFILE_READ_POLICY,
+                FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
+            .build())) {
       File dst = new File(destination.toUri());
       String lowerDst = StringUtils.toLowerCase(dst.getName());
       switch (resource.getType()) {


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


[hadoop] 04/04: HADOOP-16202. Enhanced openFile(): hadoop-azure changes. (#2584/4)

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

stevel pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 44ae2fa8e53d7d8876296642caf812f98164d27c
Author: Steve Loughran <st...@cloudera.com>
AuthorDate: Sun Apr 24 17:30:41 2022 +0100

    HADOOP-16202. Enhanced openFile(): hadoop-azure changes. (#2584/4)
    
    Stops the abfs connector warning if openFile().withFileStatus()
    is invoked with a FileStatus is not an abfs VersionedFileStatus.
    
    Contributed by Steve Loughran.
    
    Change-Id: I85076b365eb30aaef2ed35139fa8714efd4d048e
---
 .../java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java    | 2 +-
 .../hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java    | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
index 046f9f0b562..09b48a855f0 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
@@ -766,7 +766,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
         eTag = ((VersionedFileStatus) fileStatus).getVersion();
       } else {
         if (fileStatus != null) {
-          LOG.warn(
+          LOG.debug(
               "Fallback to getPathStatus REST call as provided filestatus "
                   + "is not of type VersionedFileStatus");
         }
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
index 3222a15cd52..f7fe5039799 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
@@ -37,7 +37,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_RE
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
-import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture;
+import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
 
 /**
  * Contract test for seek operation.


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