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 2019/02/05 11:57:57 UTC

[hadoop] branch trunk updated: HADOOP-15229. Add FileSystem builder-based openFile() API to match createFile(); S3A to implement S3 Select through this API.

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


The following commit(s) were added to refs/heads/trunk by this push:
     new f365957  HADOOP-15229. Add FileSystem builder-based openFile() API to match createFile(); S3A to implement S3 Select through this API.
f365957 is described below

commit f365957c6326f88734bc0a5d01cfb7eac713db20
Author: Steve Loughran <st...@apache.org>
AuthorDate: Tue Feb 5 11:51:02 2019 +0000

    HADOOP-15229. Add FileSystem builder-based openFile() API to match createFile();
    S3A to implement S3 Select through this API.
    
    The new openFile() API is asynchronous, and implemented across FileSystem and FileContext.
    
    The MapReduce V2 inputs are moved to this API, and you can actually set must/may
    options to pass in.
    
    This is more useful for setting things like s3a seek policy than for S3 select,
    as the existing input format/record readers can't handle S3 select output where
    the stream is shorter than the file length, and splitting plain text is suboptimal.
    Future work is needed there.
    
    In the meantime, any/all filesystem connectors are now free to add their own filesystem-specific
    configuration parameters which can be set in jobs and used to set filesystem input stream
    options (seek policy, retry, encryption secrets, etc).
    
    Contributed by Steve Loughran
---
 .../org/apache/hadoop/fs/AbstractFileSystem.java   |   33 +
 .../org/apache/hadoop/fs/DelegateToFileSystem.java |   20 +
 .../main/java/org/apache/hadoop/fs/FSBuilder.java  |  131 +++
 .../hadoop/fs/FSDataOutputStreamBuilder.java       |  193 +---
 .../java/org/apache/hadoop/fs/FileContext.java     |   69 +-
 .../main/java/org/apache/hadoop/fs/FileSystem.java |  188 +++-
 .../org/apache/hadoop/fs/FilterFileSystem.java     |   33 +
 .../main/java/org/apache/hadoop/fs/FilterFs.java   |   13 +
 .../hadoop/fs/FutureDataInputStreamBuilder.java    |   50 +
 .../hadoop/fs/impl/AbstractFSBuilderImpl.java      |  356 +++++++
 .../fs/impl/FutureDataInputStreamBuilderImpl.java  |  141 +++
 .../org/apache/hadoop/fs/impl/FutureIOSupport.java |  191 ++++
 .../apache/hadoop/fs/impl/WrappedIOException.java  |   56 +
 .../org/apache/hadoop/fs/impl/package-info.java    |   49 +
 .../hadoop/io/compress/PassthroughCodec.java       |  246 +++++
 .../java/org/apache/hadoop/util/LambdaUtils.java   |   59 ++
 .../src/main/resources/core-default.xml            |  112 ++
 .../src/site/markdown/filesystem/filesystem.md     |   87 +-
 .../site/markdown/filesystem/fsdatainputstream.md  |   14 +
 .../filesystem/fsdatainputstreambuilder.md         |  112 ++
 .../filesystem/fsdataoutputstreambuilder.md        |    6 +-
 .../fs/FileContextMainOperationsBaseTest.java      |   95 +-
 .../org/apache/hadoop/fs/TestHarFileSystem.java    |   20 +
 .../org/apache/hadoop/fs/TestLocalFileSystem.java  |    2 +-
 .../fs/contract/AbstractContractOpenTest.java      |  135 ++-
 .../contract/AbstractContractPathHandleTest.java   |   61 ++
 .../hadoop/fs/contract/ContractTestUtils.java      |   31 +
 .../org/apache/hadoop/test/LambdaTestUtils.java    |  130 +++
 .../apache/hadoop/test/TestLambdaTestUtils.java    |  114 +-
 .../apache/hadoop/hdfs/DistributedFileSystem.java  |    2 +-
 .../fs/contract/hdfs/TestHDFSContractOpen.java     |    2 +-
 .../org/apache/hadoop/mapred/LineRecordReader.java |   12 +-
 .../org/apache/hadoop/mapreduce/MRJobConfig.java   |   14 +
 .../lib/input/FixedLengthRecordReader.java         |   14 +-
 .../mapreduce/lib/input/LineRecordReader.java      |   12 +-
 .../mapreduce/lib/input/NLineInputFormat.java      |   12 +-
 .../hadoop-aws/dev-support/findbugs-exclude.xml    |    5 +
 .../apache/hadoop/fs/s3a/InternalConstants.java    |   53 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java    |  203 +++-
 .../org/apache/hadoop/fs/s3a/S3AInputStream.java   |   43 +-
 .../apache/hadoop/fs/s3a/S3AInstrumentation.java   |    3 +-
 .../org/apache/hadoop/fs/s3a/S3AOpContext.java     |   25 +
 .../org/apache/hadoop/fs/s3a/S3AReadOpContext.java |   97 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java    |    9 +-
 .../apache/hadoop/fs/s3a/S3ObjectAttributes.java   |   27 +-
 .../java/org/apache/hadoop/fs/s3a/Statistic.java   |    2 +
 .../apache/hadoop/fs/s3a/WriteOperationHelper.java |  105 +-
 .../apache/hadoop/fs/s3a/s3guard/S3GuardTool.java  |   82 +-
 .../fs/s3a/select/InternalSelectConstants.java     |   77 ++
 .../apache/hadoop/fs/s3a/select/SelectBinding.java |  431 ++++++++
 .../hadoop/fs/s3a/select/SelectConstants.java      |  296 ++++++
 .../hadoop/fs/s3a/select/SelectInputStream.java    |  457 ++++++++
 .../apache/hadoop/fs/s3a/select/SelectTool.java    |  355 +++++++
 .../apache/hadoop/fs/s3a/select/package-info.java  |   27 +
 .../site/markdown/tools/hadoop-aws/s3_select.md    | 1100 ++++++++++++++++++++
 .../src/site/markdown/tools/hadoop-aws/testing.md  |   16 +
 .../fs/s3a/ITestS3AAWSCredentialsProvider.java     |    4 +-
 .../hadoop/fs/s3a/ITestS3AFailureHandling.java     |   11 +-
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java     |   72 +-
 .../fs/s3a/TestS3AAWSCredentialsProvider.java      |    3 +-
 .../hadoop/fs/s3a/commit/AbstractCommitITest.java  |    2 +-
 .../s3a/s3guard/AbstractS3GuardToolTestBase.java   |   71 +-
 .../fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java   |    2 +-
 .../fs/s3a/s3guard/ITestS3GuardToolLocal.java      |   16 +-
 .../fs/s3a/s3guard/S3GuardToolTestHelper.java      |   89 ++
 .../hadoop/fs/s3a/select/AbstractS3SelectTest.java |  746 +++++++++++++
 .../org/apache/hadoop/fs/s3a/select/CsvFile.java   |  138 +++
 .../apache/hadoop/fs/s3a/select/ITestS3Select.java |  967 +++++++++++++++++
 .../hadoop/fs/s3a/select/ITestS3SelectCLI.java     |  347 ++++++
 .../hadoop/fs/s3a/select/ITestS3SelectLandsat.java |  432 ++++++++
 .../hadoop/fs/s3a/select/ITestS3SelectMRJob.java   |  206 ++++
 .../streaming/mapreduce/StreamInputFormat.java     |   14 +-
 72 files changed, 8944 insertions(+), 404 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
index 200a2d1..dc6cd2b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
@@ -25,12 +25,15 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
+import java.util.Set;
 import java.util.StringTokenizer;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -41,6 +44,7 @@ import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Options.CreateOpts;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -48,6 +52,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.LambdaUtils;
 import org.apache.hadoop.util.Progressable;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -1329,4 +1334,32 @@ public abstract class AbstractFileSystem {
     }
     return myUri.equals(((AbstractFileSystem) other).myUri);
   }
+
+  /**
+   * Open a file with the given set of options.
+   * The base implementation performs a blocking
+   * call to {@link #open(Path, int)}in this call;
+   * the actual outcome is in the returned {@code CompletableFuture}.
+   * This avoids having to create some thread pool, while still
+   * setting up the expectation that the {@code get()} call
+   * is needed to evaluate the result.
+   * @param path path to the file
+   * @param mandatoryKeys set of options declared as mandatory.
+   * @param options options set during the build sequence.
+   * @param bufferSize buffer size
+   * @return a future which will evaluate to the opened file.
+   * @throws IOException failure to resolve the link.
+   * @throws IllegalArgumentException unknown mandatory key
+   */
+  public CompletableFuture<FSDataInputStream> openFileWithOptions(Path path,
+      Set<String> mandatoryKeys,
+      Configuration options,
+      int bufferSize) throws IOException {
+    AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(mandatoryKeys,
+        Collections.emptySet(),
+        "for " + path);
+    return LambdaUtils.eval(
+        new CompletableFuture<>(), () -> open(path, bufferSize));
+  }
+
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java
index a5ab75e..165c56c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java
@@ -24,6 +24,8 @@ import java.net.URISyntaxException;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -261,4 +263,22 @@ public abstract class DelegateToFileSystem extends AbstractFileSystem {
   public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
     return Arrays.asList(fsImpl.addDelegationTokens(renewer, null));
   }
+
+  /**
+   * Open a file by delegating to
+   * {@link FileSystem#openFileWithOptions(Path, Set, Configuration, int)}.
+   * @param path path to the file
+   * @param mandatoryKeys set of options declared as mandatory.
+   * @param options options set during the build sequence.
+   * @param bufferSize buffer size
+   * @return a future which will evaluate to the opened file.
+   * @throws IOException failure to resolve the link.
+   * @throws IllegalArgumentException unknown mandatory key
+   */
+  public CompletableFuture<FSDataInputStream> openFileWithOptions(Path path,
+      Set<String> mandatoryKeys,
+      Configuration options,
+      int bufferSize) throws IOException {
+    return fsImpl.openFileWithOptions(path, mandatoryKeys, options, bufferSize);
+  }
 }
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
new file mode 100644
index 0000000..b7757a6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * The base interface which various FileSystem FileContext Builder
+ * interfaces can extend, and which underlying implementations
+ * will then implement.
+ * @param <S> Return type on the {@link #build()} call.
+ * @param <B> type of builder itself.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface FSBuilder<S, B extends FSBuilder<S, B>> {
+
+  /**
+   * Set optional Builder parameter.
+   */
+  B opt(@Nonnull String key, @Nonnull String value);
+
+  /**
+   * Set optional boolean parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  B opt(@Nonnull String key, boolean value);
+
+  /**
+   * Set optional int parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  B opt(@Nonnull String key, int value);
+
+  /**
+   * Set optional float parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  B opt(@Nonnull String key, float value);
+
+  /**
+   * Set optional double parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  B opt(@Nonnull String key, double value);
+
+  /**
+   * Set an array of string values as optional parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  B opt(@Nonnull String key, @Nonnull String... values);
+
+  /**
+   * Set mandatory option to the Builder.
+   *
+   * If the option is not supported or unavailable,
+   * the client should expect {@link #build()} throws IllegalArgumentException.
+   */
+  B must(@Nonnull String key, @Nonnull String value);
+
+  /**
+   * Set mandatory boolean option.
+   *
+   * @see #must(String, String)
+   */
+  B must(@Nonnull String key, boolean value);
+
+  /**
+   * Set mandatory int option.
+   *
+   * @see #must(String, String)
+   */
+  B must(@Nonnull String key, int value);
+
+  /**
+   * Set mandatory float option.
+   *
+   * @see #must(String, String)
+   */
+  B must(@Nonnull String key, float value);
+
+  /**
+   * Set mandatory double option.
+   *
+   * @see #must(String, String)
+   */
+  B must(@Nonnull String key, double value);
+
+  /**
+   * Set a string array as mandatory option.
+   *
+   * @see #must(String, String)
+   */
+  B must(@Nonnull String key, @Nonnull String... values);
+
+  /**
+   * Instantiate the object which was being built.
+   *
+   * @throws IllegalArgumentException if the parameters are not valid.
+   * @throws UnsupportedOperationException if the filesystem does not support
+   * the specific operation.
+   * @throws IOException on filesystem IO errors.
+   */
+  S build() throws IllegalArgumentException,
+      UnsupportedOperationException, IOException;
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
index d431293..62a3182 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
@@ -17,22 +17,18 @@
  */
 package org.apache.hadoop.fs;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
+import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Progressable;
 
 import javax.annotation.Nonnull;
 import java.io.IOException;
-import java.util.Collections;
 import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.Set;
 
+import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
 
@@ -87,9 +83,9 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public abstract class FSDataOutputStreamBuilder
-    <S extends FSDataOutputStream, B extends FSDataOutputStreamBuilder<S, B>> {
+    <S extends FSDataOutputStream, B extends FSDataOutputStreamBuilder<S, B>>
+    extends AbstractFSBuilderImpl<S, B> {
   private final FileSystem fs;
-  private final Path path;
   private FsPermission permission = null;
   private int bufferSize;
   private short replication;
@@ -101,33 +97,22 @@ public abstract class FSDataOutputStreamBuilder
   private ChecksumOpt checksumOpt = null;
 
   /**
-   * Contains optional and mandatory parameters.
-   *
-   * It does not load default configurations from default files.
-   */
-  private final Configuration options = new Configuration(false);
-
-  /** Keep track of the keys for mandatory options. */
-  private final Set<String> mandatoryKeys = new HashSet<>();
-
-  /**
    * Return the concrete implementation of the builder instance.
    */
-  protected abstract B getThisBuilder();
+  public abstract B getThisBuilder();
 
   /**
    * Construct from a {@link FileContext}.
    *
    * @param fc FileContext
    * @param p path.
-   * @throws IOException
+   * @throws IOException failure
    */
   FSDataOutputStreamBuilder(@Nonnull FileContext fc,
       @Nonnull Path p) throws IOException {
-    Preconditions.checkNotNull(fc);
-    Preconditions.checkNotNull(p);
+    super(checkNotNull(p));
+    checkNotNull(fc);
     this.fs = null;
-    this.path = p;
 
     AbstractFileSystem afs = fc.getFSofPath(p);
     FsServerDefaults defaults = afs.getServerDefaults(p);
@@ -141,25 +126,20 @@ public abstract class FSDataOutputStreamBuilder
    */
   protected FSDataOutputStreamBuilder(@Nonnull FileSystem fileSystem,
       @Nonnull Path p) {
-    Preconditions.checkNotNull(fileSystem);
-    Preconditions.checkNotNull(p);
+    super(checkNotNull(p));
+    checkNotNull(fileSystem);
     fs = fileSystem;
-    path = p;
     bufferSize = fs.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
         IO_FILE_BUFFER_SIZE_DEFAULT);
-    replication = fs.getDefaultReplication(path);
+    replication = fs.getDefaultReplication(p);
     blockSize = fs.getDefaultBlockSize(p);
   }
 
   protected FileSystem getFS() {
-    Preconditions.checkNotNull(fs);
+    checkNotNull(fs);
     return fs;
   }
 
-  protected Path getPath() {
-    return path;
-  }
-
   protected FsPermission getPermission() {
     if (permission == null) {
       permission = FsPermission.getFileDefault();
@@ -171,7 +151,7 @@ public abstract class FSDataOutputStreamBuilder
    * Set permission for the file.
    */
   public B permission(@Nonnull final FsPermission perm) {
-    Preconditions.checkNotNull(perm);
+    checkNotNull(perm);
     permission = perm;
     return getThisBuilder();
   }
@@ -235,7 +215,7 @@ public abstract class FSDataOutputStreamBuilder
    * Set the facility of reporting progress.
    */
   public B progress(@Nonnull final Progressable prog) {
-    Preconditions.checkNotNull(prog);
+    checkNotNull(prog);
     progress = prog;
     return getThisBuilder();
   }
@@ -282,155 +262,12 @@ public abstract class FSDataOutputStreamBuilder
    * Set checksum opt.
    */
   public B checksumOpt(@Nonnull final ChecksumOpt chksumOpt) {
-    Preconditions.checkNotNull(chksumOpt);
+    checkNotNull(chksumOpt);
     checksumOpt = chksumOpt;
     return getThisBuilder();
   }
 
   /**
-   * Set optional Builder parameter.
-   */
-  public B opt(@Nonnull final String key, @Nonnull final String value) {
-    mandatoryKeys.remove(key);
-    options.set(key, value);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set optional boolean parameter for the Builder.
-   *
-   * @see #opt(String, String)
-   */
-  public B opt(@Nonnull final String key, boolean value) {
-    mandatoryKeys.remove(key);
-    options.setBoolean(key, value);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set optional int parameter for the Builder.
-   *
-   * @see #opt(String, String)
-   */
-  public B opt(@Nonnull final String key, int value) {
-    mandatoryKeys.remove(key);
-    options.setInt(key, value);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set optional float parameter for the Builder.
-   *
-   * @see #opt(String, String)
-   */
-  public B opt(@Nonnull final String key, float value) {
-    mandatoryKeys.remove(key);
-    options.setFloat(key, value);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set optional double parameter for the Builder.
-   *
-   * @see #opt(String, String)
-   */
-  public B opt(@Nonnull final String key, double value) {
-    mandatoryKeys.remove(key);
-    options.setDouble(key, value);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set an array of string values as optional parameter for the Builder.
-   *
-   * @see #opt(String, String)
-   */
-  public B opt(@Nonnull final String key, @Nonnull final String... values) {
-    mandatoryKeys.remove(key);
-    options.setStrings(key, values);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set mandatory option to the Builder.
-   *
-   * If the option is not supported or unavailable on the {@link FileSystem},
-   * the client should expect {@link #build()} throws IllegalArgumentException.
-   */
-  public B must(@Nonnull final String key, @Nonnull final String value) {
-    mandatoryKeys.add(key);
-    options.set(key, value);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set mandatory boolean option.
-   *
-   * @see #must(String, String)
-   */
-  public B must(@Nonnull final String key, boolean value) {
-    mandatoryKeys.add(key);
-    options.setBoolean(key, value);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set mandatory int option.
-   *
-   * @see #must(String, String)
-   */
-  public B must(@Nonnull final String key, int value) {
-    mandatoryKeys.add(key);
-    options.setInt(key, value);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set mandatory float option.
-   *
-   * @see #must(String, String)
-   */
-  public B must(@Nonnull final String key, float value) {
-    mandatoryKeys.add(key);
-    options.setFloat(key, value);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set mandatory double option.
-   *
-   * @see #must(String, String)
-   */
-  public B must(@Nonnull final String key, double value) {
-    mandatoryKeys.add(key);
-    options.setDouble(key, value);
-    return getThisBuilder();
-  }
-
-  /**
-   * Set a string array as mandatory option.
-   *
-   * @see #must(String, String)
-   */
-  public B must(@Nonnull final String key, @Nonnull final String... values) {
-    mandatoryKeys.add(key);
-    options.setStrings(key, values);
-    return getThisBuilder();
-  }
-
-  protected Configuration getOptions() {
-    return options;
-  }
-
-  /**
-   * Get all the keys that are set as mandatory keys.
-   */
-  @VisibleForTesting
-  protected Set<String> getMandatoryKeys() {
-    return Collections.unmodifiableSet(mandatoryKeys);
-  }
-
-  /**
    * Create the FSDataOutputStream to write on the file system.
    *
    * @throws IllegalArgumentException if the parameters are not valid.
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 e5438f5..f650748 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
@@ -35,6 +35,7 @@ import java.util.Set;
 import java.util.Stack;
 import java.util.TreeSet;
 import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
 
 import javax.annotation.Nonnull;
 
@@ -44,6 +45,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.Options.CreateOpts;
+import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -56,7 +58,6 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RpcClientException;
 import org.apache.hadoop.ipc.RpcServerException;
 import org.apache.hadoop.ipc.UnexpectedServerException;
-import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -714,7 +715,7 @@ public class FileContext {
     }
 
     @Override
-    protected FCDataOutputStreamBuilder getThisBuilder() {
+    public FCDataOutputStreamBuilder getThisBuilder() {
       return this;
     }
 
@@ -2869,4 +2870,68 @@ public class FileContext {
   Tracer getTracer() {
     return tracer;
   }
+
+  /**
+   * Open a file for reading through a builder API.
+   * Ultimately calls {@link #open(Path, int)} unless a subclass
+   * executes the open command differently.
+   *
+   * The semantics of this call are therefore the same as that of
+   * {@link #open(Path, int)} with one special point: it is in
+   * {@code FSDataInputStreamBuilder.build()} in which the open operation
+   * takes place -it is there where all preconditions to the operation
+   * are checked.
+   * @param path file path
+   * @return a FSDataInputStreamBuilder object to build the input stream
+   * @throws IOException if some early checks cause IO failures.
+   * @throws UnsupportedOperationException if support is checked early.
+   */
+  @InterfaceStability.Unstable
+  public FutureDataInputStreamBuilder openFile(Path path)
+      throws IOException, UnsupportedOperationException {
+
+    return new FSDataInputStreamBuilder(path);
+  }
+
+  /**
+   * Builder returned for {@link #openFile(Path)}.
+   */
+  private class FSDataInputStreamBuilder
+      extends FutureDataInputStreamBuilderImpl {
+
+    /**
+     * Path Constructor.
+     * @param path path to open.
+     */
+    protected FSDataInputStreamBuilder(
+        @Nonnull final Path path) throws IOException {
+      super(FileContext.this, path);
+    }
+
+    /**
+     * Perform the open operation.
+     *
+     * @return a future to the input stream.
+     * @throws IOException early failure to open
+     * @throws UnsupportedOperationException if the specific operation
+     * is not supported.
+     * @throws IllegalArgumentException if the parameters are not valid.
+     */
+    @Override
+    public CompletableFuture<FSDataInputStream> build() throws IOException {
+      final Path absF = fixRelativePart(getPath());
+      return new FSLinkResolver<CompletableFuture<FSDataInputStream>>() {
+        @Override
+        public CompletableFuture<FSDataInputStream> next(
+            final AbstractFileSystem fs,
+            final Path p)
+            throws IOException {
+          return fs.openFileWithOptions(p,
+              getMandatoryKeys(),
+              getOptions(),
+              getBufferSize());
+        }
+      }.resolve(FileContext.this, absF);
+    }
+  }
 }
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 5454cd0..7e144e0 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.fs;
 
+import javax.annotation.Nonnull;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -27,6 +28,7 @@ import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -35,11 +37,13 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
+import java.util.Optional;
 import java.util.ServiceConfigurationError;
 import java.util.ServiceLoader;
 import java.util.Set;
 import java.util.Stack;
 import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
@@ -52,6 +56,8 @@ import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Options.HandleOpt;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
+import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -67,6 +73,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.DelegationTokenIssuer;
 import org.apache.hadoop.util.ClassUtil;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.LambdaUtils;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
@@ -117,6 +124,11 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
  * <li>The term "file" refers to a file in the remote filesystem,
  * rather than instances of {@code java.io.File}.</li>
  * </ol>
+ *
+ * This is a carefully evolving class.
+ * New methods may be marked as Unstable or Evolving for their initial release,
+ * as a warning that they are new and may change based on the
+ * experience of use in applications.
  *****************************************************************/
 @SuppressWarnings("DeprecatedIsStillUsed")
 @InterfaceAudience.Public
@@ -4241,6 +4253,8 @@ public abstract class FileSystem extends Configured
 
     @Override
     public FSDataOutputStream build() throws IOException {
+      rejectUnknownMandatoryKeys(Collections.emptySet(),
+          " for " + getPath());
       if (getFlags().contains(CreateFlag.CREATE) ||
           getFlags().contains(CreateFlag.OVERWRITE)) {
         if (isRecursive()) {
@@ -4255,11 +4269,12 @@ public abstract class FileSystem extends Configured
       } else if (getFlags().contains(CreateFlag.APPEND)) {
         return getFS().append(getPath(), getBufferSize(), getProgress());
       }
-      throw new IOException("Must specify either create, overwrite or append");
+      throw new PathIOException(getPath().toString(),
+          "Must specify either create, overwrite or append");
     }
 
     @Override
-    protected FileSystemDataOutputStreamBuilder getThisBuilder() {
+    public FileSystemDataOutputStreamBuilder getThisBuilder() {
       return this;
     }
   }
@@ -4287,4 +4302,173 @@ public abstract class FileSystem extends Configured
   public FSDataOutputStreamBuilder appendFile(Path path) {
     return new FileSystemDataOutputStreamBuilder(this, path).append();
   }
+
+  /**
+   * Open a file for reading through a builder API.
+   * Ultimately calls {@link #open(Path, int)} unless a subclass
+   * executes the open command differently.
+   *
+   * The semantics of this call are therefore the same as that of
+   * {@link #open(Path, int)} with one special point: it is in
+   * {@code FSDataInputStreamBuilder.build()} in which the open operation
+   * takes place -it is there where all preconditions to the operation
+   * are checked.
+   * @param path file path
+   * @return a FSDataInputStreamBuilder object to build the input stream
+   * @throws IOException if some early checks cause IO failures.
+   * @throws UnsupportedOperationException if support is checked early.
+   */
+  @InterfaceStability.Unstable
+  public FutureDataInputStreamBuilder openFile(Path path)
+      throws IOException, UnsupportedOperationException {
+    return new FSDataInputStreamBuilder(this, path).getThisBuilder();
+  }
+
+  /**
+   * Open a file for reading through a builder API.
+   * Ultimately calls {@link #open(PathHandle, int)} unless a subclass
+   * executes the open command differently.
+   *
+   * If PathHandles are unsupported, this may fail in the
+   * {@code FSDataInputStreamBuilder.build()}  command,
+   * rather than in this {@code openFile()} operation.
+   * @param pathHandle path handle.
+   * @return a FSDataInputStreamBuilder object to build the input stream
+   * @throws IOException if some early checks cause IO failures.
+   * @throws UnsupportedOperationException if support is checked early.
+   */
+  @InterfaceStability.Unstable
+  public FutureDataInputStreamBuilder openFile(PathHandle pathHandle)
+      throws IOException, UnsupportedOperationException {
+    return new FSDataInputStreamBuilder(this, pathHandle)
+        .getThisBuilder();
+  }
+
+  /**
+   * Execute the actual open file operation.
+   *
+   * This is invoked from {@code FSDataInputStreamBuilder.build()}
+   * and from {@link DelegateToFileSystem} and is where
+   * the action of opening the file should begin.
+   *
+   * The base implementation performs a blocking
+   * call to {@link #open(Path, int)}in this call;
+   * the actual outcome is in the returned {@code CompletableFuture}.
+   * This avoids having to create some thread pool, while still
+   * setting up the expectation that the {@code get()} call
+   * is needed to evaluate the result.
+   * @param path path to the file
+   * @param mandatoryKeys set of options declared as mandatory.
+   * @param options options set during the build sequence.
+   * @param bufferSize buffer size
+   * @return a future which will evaluate to the opened file.
+   * @throws IOException failure to resolve the link.
+   * @throws IllegalArgumentException unknown mandatory key
+   */
+  protected CompletableFuture<FSDataInputStream> openFileWithOptions(
+      final Path path,
+      final Set<String> mandatoryKeys,
+      final Configuration options,
+      final int bufferSize) throws IOException {
+    AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(mandatoryKeys,
+        Collections.emptySet(),
+        "for " + path);
+    return LambdaUtils.eval(
+        new CompletableFuture<>(), () -> open(path, bufferSize));
+  }
+
+  /**
+   * Execute the actual open file operation.
+   * The base implementation performs a blocking
+   * call to {@link #open(Path, int)}in this call;
+   * the actual outcome is in the returned {@code CompletableFuture}.
+   * This avoids having to create some thread pool, while still
+   * setting up the expectation that the {@code get()} call
+   * is needed to evaluate the result.
+   * @param pathHandle path to the file
+   * @param mandatoryKeys set of options declared as mandatory.
+   * @param options options set during the build sequence.
+   * @param bufferSize buffer size
+   * @return a future which will evaluate to the opened file.
+   * @throws IOException failure to resolve the link.
+   * @throws IllegalArgumentException unknown mandatory key
+   * @throws UnsupportedOperationException PathHandles are not supported.
+   * This may be deferred until the future is evaluated.
+   */
+  protected CompletableFuture<FSDataInputStream> openFileWithOptions(
+      final PathHandle pathHandle,
+      final Set<String> mandatoryKeys,
+      final Configuration options,
+      final int bufferSize) throws IOException {
+    AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(mandatoryKeys,
+        Collections.emptySet(), "");
+    CompletableFuture<FSDataInputStream> result = new CompletableFuture<>();
+    try {
+      result.complete(open(pathHandle, bufferSize));
+    } catch (UnsupportedOperationException 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;
+  }
+
+  /**
+   * Builder returned for {@code #openFile(Path)}
+   * and {@code #openFile(PathHandle)}.
+   */
+  private static class FSDataInputStreamBuilder
+      extends FutureDataInputStreamBuilderImpl
+      implements FutureDataInputStreamBuilder {
+
+    /**
+     * Path Constructor.
+     * @param fileSystem owner
+     * @param path path to open.
+     */
+    protected FSDataInputStreamBuilder(
+        @Nonnull final FileSystem fileSystem,
+        @Nonnull final Path path) {
+      super(fileSystem, path);
+    }
+
+    /**
+     * Construct from a path handle.
+     * @param fileSystem owner
+     * @param pathHandle path handle of file to open.
+     */
+    protected FSDataInputStreamBuilder(
+        @Nonnull final FileSystem fileSystem,
+        @Nonnull final PathHandle pathHandle) {
+      super(fileSystem, pathHandle);
+    }
+
+    /**
+     * Perform the open operation.
+     * Returns a future which, when get() or a chained completion
+     * operation is invoked, will supply the input stream of the file
+     * referenced by the path/path handle.
+     * @return a future to the input stream.
+     * @throws IOException early failure to open
+     * @throws UnsupportedOperationException if the specific operation
+     * is not supported.
+     * @throws IllegalArgumentException if the parameters are not valid.
+     */
+    @Override
+    public CompletableFuture<FSDataInputStream> build() throws IOException {
+      Optional<Path> optionalPath = getOptionalPath();
+      if(optionalPath.isPresent()) {
+        return getFS().openFileWithOptions(optionalPath.get(),
+            getMandatoryKeys(), getOptions(), getBufferSize());
+      } else {
+        return getFS().openFileWithOptions(getPathHandle(),
+            getMandatoryKeys(), getOptions(), getBufferSize());
+      }
+    }
+
+  }
+
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
index a3e10b4..99c18b6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
@@ -25,6 +25,8 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -692,4 +694,35 @@ public class FilterFileSystem extends FileSystem {
   public FSDataOutputStreamBuilder appendFile(Path path) {
     return fs.appendFile(path);
   }
+
+  @Override
+  public FutureDataInputStreamBuilder openFile(final Path path)
+      throws IOException, UnsupportedOperationException {
+    return fs.openFile(path);
+  }
+
+  @Override
+  public FutureDataInputStreamBuilder openFile(final PathHandle pathHandle)
+      throws IOException, UnsupportedOperationException {
+    return fs.openFile(pathHandle);
+  }
+
+  @Override
+  protected CompletableFuture<FSDataInputStream> openFileWithOptions(
+      final Path path,
+      final Set<String> mandatoryKeys,
+      final Configuration options,
+      final int bufferSize) throws IOException {
+    return fs.openFileWithOptions(path, mandatoryKeys, options, bufferSize);
+  }
+
+  @Override
+  protected CompletableFuture<FSDataInputStream> openFileWithOptions(
+      final PathHandle pathHandle,
+      final Set<String> mandatoryKeys,
+      final Configuration options,
+      final int bufferSize) throws IOException {
+    return fs.openFileWithOptions(pathHandle, mandatoryKeys, options,
+        bufferSize);
+  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java
index d6dc59b..f5430d6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java
@@ -26,9 +26,12 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -433,4 +436,14 @@ public abstract class FilterFs extends AbstractFileSystem {
       throws IOException {
     return myFs.getAllStoragePolicies();
   }
+
+  @Override
+  public CompletableFuture<FSDataInputStream> openFileWithOptions(
+      final Path path,
+      final Set<String> mandatoryKeys,
+      final Configuration options,
+      final int bufferSize) throws IOException {
+    return myFs.openFileWithOptions(path, mandatoryKeys, options, bufferSize);
+  }
+
 }
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
new file mode 100644
index 0000000..774d309
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Builder for input streams and subclasses whose return value is
+ * actually a completable future: this allows for better asynchronous
+ * operation.
+ *
+ * To be more generic, {@link #opt(String, int)} and {@link #must(String, int)}
+ * variants provide implementation-agnostic way to customize the builder.
+ * Each FS-specific builder implementation can interpret the FS-specific
+ * 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
+ * {@link IllegalArgumentException} will be thrown.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface FutureDataInputStreamBuilder
+    extends FSBuilder<CompletableFuture<FSDataInputStream>, FutureDataInputStreamBuilder> {
+
+  @Override
+  CompletableFuture<FSDataInputStream> build()
+      throws IllegalArgumentException, UnsupportedOperationException,
+      IOException;
+}
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
new file mode 100644
index 0000000..5fc92e9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import javax.annotation.Nonnull;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.Set;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSBuilder;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Builder for filesystem/filecontext operations of various kinds,
+ * with option support.
+ *
+ * <code>
+ *   .opt("foofs:option.a", true)
+ *   .opt("foofs:option.b", "value")
+ *   .opt("barfs:cache", true)
+ *   .must("foofs:cache", true)
+ *   .must("barfs:cache-size", 256 * 1024 * 1024)
+ *   .build();
+ * </code>
+ *
+ * Configuration keys declared in an {@code opt()} may be ignored by
+ * a builder which does not recognise them.
+ *
+ * Configuration keys declared in a {@code must()} function set must
+ * be understood by the implementation or a
+ * {@link IllegalArgumentException} will be thrown.
+ *
+ * @param <S> Return type on the {@link #build()} call.
+ * @param <B> type of builder itself.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class
+    AbstractFSBuilderImpl<S, B extends FSBuilder<S, B>>
+    implements FSBuilder<S, B> {
+
+  public static final String UNKNOWN_MANDATORY_KEY = "Unknown mandatory key";
+
+  @VisibleForTesting
+  static final String E_BOTH_A_PATH_AND_A_PATH_HANDLE
+      = "Both a path and a pathHandle has been provided to the constructor";
+
+  private final Optional<Path> optionalPath;
+
+  private final Optional<PathHandle> optionalPathHandle;
+
+  /**
+   * Contains optional and mandatory parameters.
+   *
+   * It does not load default configurations from default files.
+   */
+  private final Configuration options = new Configuration(false);
+
+  /** Keep track of the keys for mandatory options. */
+  private final Set<String> mandatoryKeys = new HashSet<>();
+
+  /**
+   * Constructor with both optional path and path handle.
+   * Either or both argument may be empty, but it is an error for
+   * both to be defined.
+   * @param optionalPath a path or empty
+   * @param optionalPathHandle a path handle/empty
+   * @throws IllegalArgumentException if both parameters are set.
+   */
+  protected AbstractFSBuilderImpl(
+      @Nonnull Optional<Path> optionalPath,
+      @Nonnull Optional<PathHandle> optionalPathHandle) {
+    checkArgument(!(checkNotNull(optionalPath).isPresent()
+            && checkNotNull(optionalPathHandle).isPresent()),
+        E_BOTH_A_PATH_AND_A_PATH_HANDLE);
+    this.optionalPath = optionalPath;
+    this.optionalPathHandle = optionalPathHandle;
+  }
+
+  protected AbstractFSBuilderImpl(@Nonnull final Path path) {
+    this(Optional.of(path), Optional.empty());
+  }
+
+  protected AbstractFSBuilderImpl(@Nonnull final PathHandle pathHandle) {
+    this(Optional.empty(), Optional.of(pathHandle));
+  }
+
+
+  /**
+   * Get the cast builder.
+   * @return this object, typecast
+   */
+  public B getThisBuilder() {
+    return (B)this;
+  }
+
+  /**
+   * Get the optional path; may be empty.
+   * @return the optional path field.
+   */
+  public Optional<Path> getOptionalPath() {
+    return optionalPath;
+  }
+
+  /**
+   * Get the path: only valid if constructed with a path.
+   * @return the path
+   * @throws NoSuchElementException if the field is empty.
+   */
+  public Path getPath() {
+    return optionalPath.get();
+  }
+
+  /**
+   * Get the optional path handle; may be empty.
+   * @return the optional path handle field.
+   */
+  public Optional<PathHandle> getOptionalPathHandle() {
+    return optionalPathHandle;
+  }
+
+  /**
+   * Get the PathHandle: only valid if constructed with a PathHandle.
+   * @return the PathHandle
+   * @throws NoSuchElementException if the field is empty.
+   */
+  public PathHandle getPathHandle() {
+    return optionalPathHandle.get();
+  }
+
+  /**
+   * Set optional Builder parameter.
+   */
+  @Override
+  public B opt(@Nonnull final String key, @Nonnull final String value) {
+    mandatoryKeys.remove(key);
+    options.set(key, value);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set optional boolean parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  @Override
+  public B opt(@Nonnull final String key, boolean value) {
+    mandatoryKeys.remove(key);
+    options.setBoolean(key, value);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set optional int parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  @Override
+  public B opt(@Nonnull final String key, int value) {
+    mandatoryKeys.remove(key);
+    options.setInt(key, value);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set optional float parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  @Override
+  public B opt(@Nonnull final String key, float value) {
+    mandatoryKeys.remove(key);
+    options.setFloat(key, value);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set optional double parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  @Override
+  public B opt(@Nonnull final String key, double value) {
+    mandatoryKeys.remove(key);
+    options.setDouble(key, value);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set an array of string values as optional parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  @Override
+  public B opt(@Nonnull final String key, @Nonnull final String... values) {
+    mandatoryKeys.remove(key);
+    options.setStrings(key, values);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set mandatory option to the Builder.
+   *
+   * If the option is not supported or unavailable on the {@link FileSystem},
+   * the client should expect {@link #build()} throws IllegalArgumentException.
+   */
+  @Override
+  public B must(@Nonnull final String key, @Nonnull final String value) {
+    mandatoryKeys.add(key);
+    options.set(key, value);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set mandatory boolean option.
+   *
+   * @see #must(String, String)
+   */
+  @Override
+  public B must(@Nonnull final String key, boolean value) {
+    mandatoryKeys.add(key);
+    options.setBoolean(key, value);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set mandatory int option.
+   *
+   * @see #must(String, String)
+   */
+  @Override
+  public B must(@Nonnull final String key, int value) {
+    mandatoryKeys.add(key);
+    options.setInt(key, value);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set mandatory float option.
+   *
+   * @see #must(String, String)
+   */
+  @Override
+  public B must(@Nonnull final String key, float value) {
+    mandatoryKeys.add(key);
+    options.setFloat(key, value);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set mandatory double option.
+   *
+   * @see #must(String, String)
+   */
+  @Override
+  public B must(@Nonnull final String key, double value) {
+    mandatoryKeys.add(key);
+    options.setDouble(key, value);
+    return getThisBuilder();
+  }
+
+  /**
+   * Set a string array as mandatory option.
+   *
+   * @see #must(String, String)
+   */
+  @Override
+  public B must(@Nonnull final String key, @Nonnull final String... values) {
+    mandatoryKeys.add(key);
+    options.setStrings(key, values);
+    return getThisBuilder();
+  }
+
+  /**
+   * Get the mutable option configuration.
+   * @return the option configuration.
+   */
+  public Configuration getOptions() {
+    return options;
+  }
+
+  /**
+   * Get all the keys that are set as mandatory keys.
+   */
+  public Set<String> getMandatoryKeys() {
+    return Collections.unmodifiableSet(mandatoryKeys);
+  }
+
+  /**
+   * Reject a configuration if one or more mandatory keys are
+   * not in the set of mandatory keys.
+   * The first invalid key raises the exception; the order of the
+   * scan and hence the specific key raising the exception is undefined.
+   * @param knownKeys a possibly empty collection of known keys
+   * @param extraErrorText extra error text to include.
+   * @throws IllegalArgumentException if any key is unknown.
+   */
+  protected void rejectUnknownMandatoryKeys(final Collection<String> knownKeys,
+      String extraErrorText)
+      throws IllegalArgumentException {
+    rejectUnknownMandatoryKeys(mandatoryKeys, knownKeys, extraErrorText);
+  }
+
+  /**
+   * Reject a configuration if one or more mandatory keys are
+   * not in the set of mandatory keys.
+   * The first invalid key raises the exception; the order of the
+   * scan and hence the specific key raising the exception is undefined.
+   * @param mandatory the set of mandatory keys
+   * @param knownKeys a possibly empty collection of known keys
+   * @param extraErrorText extra error text to include.
+   * @throws IllegalArgumentException if any key is unknown.
+   */
+  public static void rejectUnknownMandatoryKeys(
+      final Set<String> mandatory,
+      final Collection<String> knownKeys,
+      final String extraErrorText)
+      throws IllegalArgumentException {
+    final String eText = extraErrorText.isEmpty()
+        ? ""
+        : (extraErrorText + " ");
+    mandatory.forEach((key) ->
+        checkArgument(knownKeys.contains(key),
+            UNKNOWN_MANDATORY_KEY + " %s\"%s\"", eText, key));
+  }
+
+}
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
new file mode 100644
index 0000000..2aa4a5d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
+/**
+ * Builder for input streams and subclasses whose return value is
+ * actually a completable future: this allows for better asynchronous
+ * operation.
+ *
+ * To be more generic, {@link #opt(String, int)} and {@link #must(String, int)}
+ * variants provide implementation-agnostic way to customize the builder.
+ * Each FS-specific builder implementation can interpret the FS-specific
+ * 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
+ * {@link IllegalArgumentException} will be thrown.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class FutureDataInputStreamBuilderImpl
+    extends AbstractFSBuilderImpl<CompletableFuture<FSDataInputStream>, FutureDataInputStreamBuilder>
+    implements FutureDataInputStreamBuilder {
+
+  private final FileSystem fileSystem;
+
+  private int bufferSize;
+
+  /**
+   * Construct from a {@link FileContext}.
+   *
+   * @param fc FileContext
+   * @param path path.
+   * @throws IOException failure
+   */
+  protected FutureDataInputStreamBuilderImpl(@Nonnull FileContext fc,
+      @Nonnull Path path) throws IOException {
+    super(checkNotNull(path));
+    checkNotNull(fc);
+    this.fileSystem = null;
+    bufferSize = IO_FILE_BUFFER_SIZE_DEFAULT;
+  }
+
+  /**
+   * Constructor.
+   * @param fileSystem owner FS.
+   * @param path path
+   */
+  protected FutureDataInputStreamBuilderImpl(@Nonnull FileSystem fileSystem,
+      @Nonnull Path path) {
+    super(checkNotNull(path));
+    this.fileSystem = checkNotNull(fileSystem);
+    initFromFS();
+  }
+
+  /**
+   * Constructor with PathHandle.
+   * @param fileSystem owner FS.
+   * @param pathHandle path handle
+   */
+  public FutureDataInputStreamBuilderImpl(@Nonnull FileSystem fileSystem,
+      @Nonnull PathHandle pathHandle) {
+    super(pathHandle);
+    this.fileSystem = fileSystem;
+    initFromFS();
+  }
+
+  /**
+   * Initialize from a filesystem.
+   */
+  private void initFromFS() {
+    bufferSize = fileSystem.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
+        IO_FILE_BUFFER_SIZE_DEFAULT);
+  }
+
+  protected FileSystem getFS() {
+    checkNotNull(fileSystem);
+    return fileSystem;
+  }
+
+  protected int getBufferSize() {
+    return bufferSize;
+  }
+
+  /**
+   * Set the size of the buffer to be used.
+   */
+  public FutureDataInputStreamBuilder bufferSize(int bufSize) {
+    bufferSize = bufSize;
+    return getThisBuilder();
+  }
+
+  /**
+   * Get the builder.
+   * This must be used after the constructor has been invoked to create
+   * the actual builder: it allows for subclasses to do things after
+   * construction.
+   */
+  public FutureDataInputStreamBuilder builder() {
+    return getThisBuilder();
+  }
+
+  @Override
+  public FutureDataInputStreamBuilder getThisBuilder() {
+    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
new file mode 100644
index 0000000..9d5f2bf
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+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;
+
+/**
+ * Support for future IO and the FS Builder subclasses.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class FutureIOSupport {
+
+  private FutureIOSupport() {
+  }
+
+  /**
+   * Given a future, evaluate it. Raised exceptions are
+   * extracted and handled.
+   * @param future future to evaluate
+   * @param <T> type of the result.
+   * @return the result, if all went well.
+   * @throws InterruptedIOException future was interrupted
+   * @throws IOException if something went wrong
+   * @throws RuntimeException any nested RTE thrown
+   */
+  public static <T> T awaitFuture(final Future<T> future)
+      throws InterruptedIOException, IOException, RuntimeException {
+    try {
+      return future.get();
+    } catch (InterruptedException e) {
+      throw (InterruptedIOException)new InterruptedIOException(e.toString())
+          .initCause(e);
+    } catch (ExecutionException e) {
+      return raiseInnerCause(e);
+    }
+  }
+
+
+  /**
+   * Given a future, evaluate it. Raised exceptions are
+   * extracted and handled.
+   * @param future future to evaluate
+   * @param <T> type of the result.
+   * @return the result, if all went well.
+   * @throws InterruptedIOException future was interrupted
+   * @throws IOException if something went wrong
+   * @throws RuntimeException any nested RTE thrown
+   * @throws TimeoutException the future timed out.
+   */
+  public static <T> T awaitFuture(final Future<T> future,
+      final long timeout,
+      final TimeUnit unit)
+      throws InterruptedIOException, IOException, RuntimeException,
+      TimeoutException {
+
+    try {
+      return future.get(timeout, unit);
+    } catch (InterruptedException e) {
+      throw (InterruptedIOException)new InterruptedIOException(e.toString())
+          .initCause(e);
+    } catch (ExecutionException e) {
+      return raiseInnerCause(e);
+    }
+  }
+
+
+  /**
+   * 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.
+   *
+   * @param e exception.
+   * @param <T> type of return value.
+   * @return nothing, ever.
+   * @throws IOException either the inner IOException, or a wrapper around
+   * any non-Runtime-Exception
+   * @throws RuntimeException if that is the inner cause.
+   */
+  public static <T> T raiseInnerCause(final ExecutionException e)
+      throws IOException {
+    Throwable cause = e.getCause();
+    if (cause instanceof IOException) {
+      throw (IOException) cause;
+    } else if (cause instanceof WrappedIOException){
+      throw ((WrappedIOException) cause).getCause();
+    } else if (cause instanceof RuntimeException){
+      throw (RuntimeException) cause;
+    } else if (cause != null) {
+      // other type: wrap with a new IOE
+      throw new IOException(cause);
+    } else {
+      // this only happens if somebody deliberately raises
+      // an ExecutionException
+      throw new IOException(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>
+   *   fs.example.s3a.option => s3a:option
+   *   fs.example.fs.io.policy => s3a.io.policy
+   *   fs.example.something => something
+   * </pre>
+   * @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 => s3a:option
+   *   fs.example.fs.io.policy => s3a.io.policy
+   *   fs.example.something => 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);
+      }
+    }
+  }
+}
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
new file mode 100644
index 0000000..1de1ecb
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/WrappedIOException.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.impl;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.base.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.
+ *
+ * The constructor signature guarantees the cause will be an IOException,
+ * and as it checks for a null-argument, non-null.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class WrappedIOException extends RuntimeException {
+
+  private static final long serialVersionUID = 2510210974235779294L;
+
+  /**
+   * Construct from a non-null IOException.
+   * @param cause inner cause
+   * @throws NullPointerException if the cause is null.
+   */
+  public WrappedIOException(final IOException cause) {
+    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/impl/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/package-info.java
new file mode 100644
index 0000000..f1cd76c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/package-info.java
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains implementation classes for use inside
+ * filesystems.
+ *
+ * These classes MUST NOT be directly exposed as the arguments
+ * or return values of methods, or as part of a visible
+ * inheritance tree.
+ *
+ * These classes MAY be returned behind interfaces.
+ * When such interfaces are used as parameters, the methods
+ * which accept the interfaces MUST NOT cast them to the classes
+ * contained therein: they MUST interact purely through
+ * the interface.
+ *
+ * That is: don't expose the implementation classes in here,
+ * and don't expect input interface implementations to always
+ * be the classes in here.
+ *
+ * These classes are for the private use of FileSystem/
+ * FileContext implementations.
+ * Implementation classes not developed within the ASF Hadoop
+ * codebase MAY use these, with the caveat that these classes
+ * are highly unstable.
+ */
+
+@InterfaceAudience.LimitedPrivate("Filesystems")
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/PassthroughCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/PassthroughCodec.java
new file mode 100644
index 0000000..a3f0bff
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/PassthroughCodec.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.compress;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * This is a special codec which does not transform the output.
+ * It can be declared as a codec in the option "io.compression.codecs",
+ * and then it will declare that it supports the file extension
+ * set in {@link #OPT_EXTENSION}.
+ *
+ * This allows decompression to be disabled on a job, even when there is
+ * a registered/discoverable decompression codec for a file extension
+ * -without having to change the standard codec binding mechanism.
+ *
+ * For example, to disable decompression for a gzipped files, set the
+ * options
+ * <pre>
+ *   io.compression.codecs = org.apache.hadoop.io.compress.PassthroughCodec
+ *   io.compress.passthrough.extension = .gz
+ * </pre>
+ *
+ * <i>Note:</i> this is not a Splittable codec: it doesn't know the
+ * capabilities of the passed in stream. It should be possible to
+ * extend this in a subclass: the inner classes are marked as protected
+ * to enable this. <i>Do not retrofit splitting to this class.</i>.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class PassthroughCodec
+    implements Configurable, CompressionCodec {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PassthroughCodec.class);
+
+  /**
+   * Classname of the codec: {@value}.
+   */
+  public static final String CLASSNAME =
+      "org.apache.hadoop.io.compress.PassthroughCodec";
+
+  /**
+   * Option to control the extension of the code: {@value}.
+   */
+  public static final String OPT_EXTENSION =
+      "io.compress.passthrough.extension";
+
+  /**
+   * This default extension is here so that if no extension has been defined,
+   * some value is still returned: {@value}..
+   */
+  public static final String DEFAULT_EXTENSION = ".passthrough";
+
+  private Configuration conf;
+
+  private String extension = DEFAULT_EXTENSION;
+
+  public PassthroughCodec() {
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(final Configuration conf) {
+    this.conf = conf;
+    // update the default extension value at this point, adding
+    // a dot prefix if needed.
+    String ex = conf.getTrimmed(OPT_EXTENSION, DEFAULT_EXTENSION);
+    extension = ex.startsWith(".") ? ex : ("." + ex);
+  }
+
+  @Override
+  public String getDefaultExtension() {
+    LOG.info("Registering fake codec for extension {}", extension);
+    return extension;
+  }
+
+  @Override
+  public CompressionOutputStream createOutputStream(final OutputStream out)
+      throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public CompressionOutputStream createOutputStream(final OutputStream out,
+      final Compressor compressor) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Class<? extends Compressor> getCompressorType() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Compressor createCompressor() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public CompressionInputStream createInputStream(final InputStream in)
+      throws IOException {
+    return createInputStream(in, null);
+  }
+
+  @Override
+  public CompressionInputStream createInputStream(final InputStream in,
+      final Decompressor decompressor) throws IOException {
+    return new PassthroughDecompressorStream(in);
+  }
+
+  @Override
+  public Class<? extends Decompressor> getDecompressorType() {
+    return StubDecompressor.class;
+  }
+
+  @Override
+  public Decompressor createDecompressor() {
+    return new StubDecompressor();
+  }
+
+  /**
+   * The decompressor.
+   */
+  protected static final class PassthroughDecompressorStream
+      extends DecompressorStream {
+
+    private final InputStream input;
+
+    PassthroughDecompressorStream(final InputStream input)
+        throws IOException {
+      super(input);
+      this.input = input;
+    }
+
+    @Override
+    public int read(final byte[] b) throws IOException {
+      return input.read(b);
+    }
+
+    @Override
+    public int read() throws IOException {
+      return input.read();
+    }
+
+    @Override
+    public int read(final byte[] b, final int off, final int len)
+        throws IOException {
+      return input.read(b, off, len);
+    }
+
+    @Override
+    public long skip(final long n) throws IOException {
+      return input.skip(n);
+    }
+
+    @Override
+    public int available() throws IOException {
+      return input.available();
+    }
+  }
+
+  /**
+   * The decompressor is a no-op. It is not needed other than
+   * to complete the methods offered by the interface.
+   */
+  protected static final class StubDecompressor implements Decompressor {
+
+    @Override
+    public void setInput(final byte[] b, final int off, final int len) {
+
+    }
+
+    @Override
+    public boolean needsInput() {
+      return false;
+    }
+
+    @Override
+    public void setDictionary(final byte[] b, final int off, final int len) {
+
+    }
+
+    @Override
+    public boolean needsDictionary() {
+      return false;
+    }
+
+    @Override
+    public boolean finished() {
+      return false;
+    }
+
+    @Override
+    public int decompress(final byte[] b, final int off, final int len)
+        throws IOException {
+      return 0;
+    }
+
+    @Override
+    public int getRemaining() {
+      return 0;
+    }
+
+    @Override
+    public void reset() {
+
+    }
+
+    @Override
+    public void end() {
+
+    }
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LambdaUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LambdaUtils.java
new file mode 100644
index 0000000..14c6db6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LambdaUtils.java
@@ -0,0 +1,59 @@
+/*
+ * 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.util;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Lambda-expression utilities be they generic or specific to
+ * Hadoop datatypes.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class LambdaUtils {
+
+  private LambdaUtils() {
+  }
+
+  /**
+   * Utility method to evaluate a callable and fill in the future
+   * with the result or the exception raised.
+   * Once this method returns, the future will have been evaluated to
+   * either a return value or an exception.
+   * @param <T> type of future
+   * @param result future for the result.
+   * @param call callable to invoke.
+   * @return the future passed in
+   */
+  public static <T> CompletableFuture<T> eval(
+      final CompletableFuture<T> result,
+      final Callable<T> call) {
+    try {
+      result.complete(call.call());
+    } catch (Throwable tx) {
+      result.completeExceptionally(tx);
+    }
+    return result;
+  }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 73f2d10..d05e1bb 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1713,6 +1713,118 @@
 </property>
 
 <property>
+  <name>fs.s3a.select.enabled</name>
+  <value>true</value>
+  <description>Is S3 Select enabled?</description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.comment.marker</name>
+  <value>#</value>
+  <description>In S3 Select queries: the marker for comment lines in CSV files</description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.record.delimiter</name>
+  <value>\n</value>
+  <description>In S3 Select queries over CSV files: the record delimiter.
+    \t is remapped to the TAB character, \r to CR \n to newline. \\ to \
+    and \" to "
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.field.delimiter</name>
+  <value>,</value>
+  <description>In S3 Select queries over CSV files: the field delimiter.
+    \t is remapped to the TAB character, \r to CR \n to newline. \\ to \
+    and \" to "
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.quote.character</name>
+  <value>"</value>
+  <description>In S3 Select queries over CSV files: quote character.
+    \t is remapped to the TAB character, \r to CR \n to newline. \\ to \
+    and \" to "
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.quote.escape.character</name>
+  <value>\\</value>
+  <description>In S3 Select queries over CSV files: quote escape character.
+    \t is remapped to the TAB character, \r to CR \n to newline. \\ to \
+    and \" to "
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.header</name>
+  <value>none</value>
+  <description>In S3 Select queries over CSV files: what is the role of the header? One of "none", "ignore" and "use"</description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.compression</name>
+  <value>none</value>
+  <description>In S3 Select queries, the source compression
+    algorithm. One of: "none" and "gzip"</description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.csv.quote.fields</name>
+  <value>always</value>
+  <description>
+    In S3 Select queries: should fields in generated CSV Files be quoted?
+    One of: "always", "asneeded".
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.csv.quote.character</name>
+  <value>"</value>
+  <description>
+    In S3 Select queries: the quote character for generated CSV Files.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.csv.quote.escape.character</name>
+  <value>\\</value>
+  <description>
+    In S3 Select queries: the quote escape character for generated CSV Files.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.csv.record.delimiter</name>
+  <value>\n</value>
+  <description>
+    In S3 Select queries: the record delimiter for generated CSV Files.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.csv.field.delimiter</name>
+  <value>,</value>
+  <description>
+    In S3 Select queries: the field delimiter for generated CSV Files.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.errors.include.sql</name>
+  <value>false</value>
+  <description>
+    Include the SQL statement in errors: this is useful for development but
+    may leak security and Personally Identifying Information in production,
+    so must be disabled there.
+  </description>
+</property>
+
+<property>
   <name>fs.AbstractFileSystem.s3a.impl</name>
   <value>org.apache.hadoop.fs.s3a.S3A</value>
   <description>The implementation class of the S3A AbstractFileSystem.</description>
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 28c6fbe..3751847 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
@@ -693,9 +693,94 @@ symbolic links
 exists in the metadata, but no copies of any its blocks can be located;
 -`FileNotFoundException` would seem more accurate and useful.
 
+### `FSDataInputStreamBuilder openFile(Path path)`
+
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(Path, Set<String>, Configuration, int)` 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, Set<String>, Configuration, int)`
+ultimately invokes `open(Path, int)`.
+
+Thus the chain `openFile(path).build().get()` has the same preconditions
+and postconditions as `open(Path p, int bufferSize)`
+
+
+The `openFile()` operation may check the state of the filesystem during this
+call, 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.
+
+### `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, Set<String>, Configuration, int)` 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(Path,PathHandle, Set<String>, Configuration, int)` 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.
+
 ### `PathHandle getPathHandle(FileStatus stat, HandleOpt... options)`
 
-Implementaions without a compliant call MUST throw `UnsupportedOperationException`
+Implementations without a compliant call MUST throw `UnsupportedOperationException`
 
 #### Preconditions
 
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
index 32eeb5b..e067b07 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md
@@ -200,6 +200,10 @@ Some FileSystems do not raise an exception if this condition is not met. They
 instead return -1 on any `read()` operation where, at the time of the read,
 `len(data(FSDIS)) < pos(FSDIS)`.
 
+After a failed seek, the value of `pos(FSDIS)` may change.
+As an example, seeking past the EOF may move the read position
+to the end of the file, *as well as raising an `EOFException`.*
+
 #### Postconditions
 
     FSDIS' = (s, data, True)
@@ -211,6 +215,16 @@ There is an implicit invariant: a seek to the current position is a no-op
 Implementations may recognise this operation and bypass all other precondition
 checks, leaving the input stream unchanged.
 
+The most recent connectors to object stores all implement some form
+of "lazy-seek": the `seek()` call may appear to update the stream, and the value
+of `getPos()` is updated, but the file is not opened/reopenend until
+data is actually read. Implementations of lazy seek MUST still validate
+the new seek position against the known length of the file.
+However the state of the file (i.e. does it exist, what
+its current length is) does not need to be refreshed at this point.
+The fact that a file has been deleted or truncated may not surface until
+that `read()` call.
+
 
 ### `Seekable.seekToNewSource(offset)`
 
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
new file mode 100644
index 0000000..f1beed8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md
@@ -0,0 +1,112 @@
+<!---
+  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.
+-->
+
+<!--  ============================================================= -->
+<!--  CLASS: FSDataInputStreamBuilder -->
+<!--  ============================================================= -->
+
+# class `org.apache.hadoop.fs.FSDataInputStreamBuilder`
+
+<!-- MACRO{toc|fromDepth=1|toDepth=2} -->
+
+An interface offering of the Builder pattern for creating Java `Future`
+references to `FSDataInputStream` and its subclasses.
+It is used to initate a (potentially asynchronous) operation to open an existing
+file for reading.
+
+## Invariants
+
+The `FSDataInputStreamBuilder` interface does not require parameters or
+or the state of `FileSystem` until [`build()`](#build) is
+invoked and/or during the asynchronous open operation itself.
+
+Some aspects of the state of the filesystem, MAY be checked in the initial
+`openFile()` call, provided they are known to be invariants which will not
+change between `openFile()` and the `build().get()` sequence. For example,
+path validation.
+
+## Implementation-agnostic parameters.
+
+
+### <a name="Builder.bufferSize"></a> `FSDataInputStreamBuilder bufferSize(int bufSize)`
+
+Set the size of the buffer to be used.
+
+### Set optional or mandatory parameters
+
+    FSDataInputStreamBuilder opt(String key, ...)
+    FSDataInputStreamBuilder 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`.
+
+```java
+out = fs.openFile(path)
+    .opt("fs.s3a.experimental.fadvise", "random")
+    .must("fs.s3a.readahead.range", 256 * 1024)
+    .build()
+    .get();
+```
+
+#### Implementation Notes
+
+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
+a feature which is recognized but not supported in the specific
+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:
+
+> The last option specified defines the value and its optional/mandatory state.
+
+
+## Builder interface
+
+### <a name="build"></a> `CompletableFuture<FSDataInputStream> build()`
+
+
+Return an `CompletableFuture<FSDataInputStream>` which, when successfully
+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`
+
+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.
+
+Thus, if even a file does not exist, the following call will still succeed, returning
+a future to be evaluated.
+
+```java
+Path p = new Path("file://tmp/file-which-does-not-exist");
+
+CompletableFuture<FSDataInputStream> future = p.getFileSystem(conf)
+      .openFile(p)
+      .build;
+```
+
+The preconditions for opening the file are checked during the asynchronous
+evaluation, and so will surface when the future is completed:
+
+```java
+FSDataInputStream in = future.get();
+```
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md
index 4ea1fd1..64dda2d 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md
@@ -114,10 +114,12 @@ MUST verify that implementation-agnostic parameters (i.e., "syncable") or
 implementation-specific parameters (i.e., "foofs:cache")
 are supported. `FileSystem` will satisfy optional parameters (via `opt(key, ...)`)
 on best effort. If the mandatory parameters (via `must(key, ...)`) can not be satisfied
-in the `FileSystem`, `IllegalArgumentException` should be thrown in `build()`.
+in the `FileSystem`, `IllegalArgumentException` must be thrown in `build()`.
 
 The behavior of resolving the conflicts between the parameters set by
-builder methods (i.e., `bufferSize()`) and `opt()`/`must()` is undefined.
+builder methods (i.e., `bufferSize()`) and `opt()`/`must()` is as follows:
+
+> The last option specified defines the value and its optional/mandatory state.
 
 ## HDFS-specific parameters.
 
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
index c07a6ff..4c90490 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
@@ -23,10 +23,13 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.EnumSet;
 import java.util.NoSuchElementException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.fs.Options.CreateOpts;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -40,6 +43,8 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 import static org.apache.hadoop.fs.CreateFlag.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
 
 /**
  * <p>
@@ -1326,13 +1331,10 @@ public abstract class FileContextMainOperationsBaseTest  {
     final Path path = new Path(rootPath, "zoo");
     createFile(path);
     final long length = fc.getFileStatus(path).getLen();
-    FSDataInputStream fsdis = fc.open(path, 2048);
-    try {
-      byte[] bb = new byte[(int)length];
+    try (FSDataInputStream fsdis = fc.open(path, 2048)) {
+      byte[] bb = new byte[(int) length];
       fsdis.readFully(bb);
       assertArrayEquals(data, bb);
-    } finally {
-      fsdis.close();
     }
   }
 
@@ -1452,4 +1454,87 @@ public abstract class FileContextMainOperationsBaseTest  {
   private Path getTestRootPath(FileContext fc, String pathString) {
     return fileContextTestHelper.getTestRootPath(fc, pathString);
   }
+
+  /**
+   * Create a path under the test path.
+   * @param filepath path string in
+   * @return a path qualified by the test filesystem
+   * @throws IOException IO problems
+   */
+  protected Path path(String filepath) throws IOException {
+    return getTestRootPath(fc, filepath);
+  }
+
+  /**
+   * Describe a test. This is a replacement for javadocs
+   * where the tests role is printed in the log output
+   * @param text description
+   */
+  protected void describe(String text) {
+    LOG.info(text);
+  }
+
+  @Test
+  public void testOpenFileRead() throws Exception {
+    final Path path = path("testOpenFileRead");
+    createFile(path);
+    final long length = fc.getFileStatus(path).getLen();
+    try (FSDataInputStream fsdis = fc.openFile(path)
+        .opt("fs.test.something", true)
+        .opt("fs.test.something2", 3)
+        .opt("fs.test.something3", "3")
+        .build().get()) {
+      byte[] bb = new byte[(int) length];
+      fsdis.readFully(bb);
+      assertArrayEquals(data, bb);
+    }
+  }
+
+  @Test
+  public void testOpenFileUnknownOption() throws Throwable {
+    describe("calling openFile fails when a 'must()' option is unknown");
+
+    final Path path = path("testOpenFileUnknownOption");
+    FutureDataInputStreamBuilder builder =
+        fc.openFile(path)
+            .opt("fs.test.something", true)
+            .must("fs.test.something", true);
+    intercept(IllegalArgumentException.class,
+        () -> builder.build());
+  }
+
+  @Test
+  public void testOpenFileLazyFail() throws Throwable {
+    describe("openFile fails on a missing file in the get() and not before");
+    FutureDataInputStreamBuilder builder =
+        fc.openFile(path("testOpenFileUnknownOption"))
+            .opt("fs.test.something", true);
+    interceptFuture(FileNotFoundException.class, "", builder.build());
+  }
+
+  @Test
+  public void testOpenFileApplyRead() throws Throwable {
+    describe("use the apply sequence");
+    Path path = path("testOpenFileApplyRead");
+    createFile(path);
+    CompletableFuture<Long> readAllBytes = fc.openFile(path)
+        .build()
+        .thenApply(ContractTestUtils::readStream);
+    assertEquals("Wrong number of bytes read from stream",
+        data.length,
+        (long)readAllBytes.get());
+  }
+
+  @Test
+  public void testOpenFileApplyAsyncRead() throws Throwable {
+    describe("verify that async accept callbacks are evaluated");
+    Path path = path("testOpenFileApplyAsyncRead");
+    createFile(path);
+    CompletableFuture<FSDataInputStream> future = fc.openFile(path).build();
+    AtomicBoolean accepted = new AtomicBoolean(false);
+    future.thenAcceptAsync(i -> accepted.set(true)).get();
+    assertTrue("async accept operation not invoked",
+        accepted.get());
+  }
+
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
index b3021a9..57798c2 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
@@ -40,6 +40,8 @@ import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
 
 import static org.apache.hadoop.fs.Options.ChecksumOpt;
 import static org.apache.hadoop.fs.Options.CreateOpts;
@@ -230,6 +232,24 @@ public class TestHarFileSystem {
 
     public Collection<FileStatus> getTrashRoots(boolean allUsers) throws IOException;
     StorageStatistics getStorageStatistics();
+
+    FutureDataInputStreamBuilder openFile(Path path)
+        throws IOException, UnsupportedOperationException;
+
+    FutureDataInputStreamBuilder openFile(PathHandle pathHandle)
+        throws IOException, UnsupportedOperationException;
+
+    CompletableFuture<FSDataInputStream> openFileWithOptions(
+        PathHandle pathHandle,
+        Set<String> mandatoryKeys,
+        Configuration options,
+        int bufferSize) throws IOException;
+
+    CompletableFuture<FSDataInputStream> openFileWithOptions(
+        Path path,
+        Set<String> mandatoryKeys,
+        Configuration options,
+        int bufferSize) throws IOException;
   }
 
   @Test
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
index d5622af..fae3db8 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
@@ -729,7 +729,7 @@ public class TestLocalFileSystem {
     }
 
     @Override
-    protected BuilderWithSupportedKeys getThisBuilder() {
+    public BuilderWithSupportedKeys getThisBuilder() {
       return this;
     }
 
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 d475c6e..c9283dc 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
@@ -19,22 +19,30 @@ package org.apache.hadoop.fs.contract;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 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.impl.FutureIOSupport;
 import org.apache.hadoop.io.IOUtils;
 
 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 org.junit.Test;
 
 /**
- * Test Seek operations
+ * Test Open operations.
  */
 public abstract class AbstractContractOpenTest
     extends AbstractFSContractTestBase {
@@ -63,8 +71,7 @@ public abstract class AbstractContractOpenTest
     instream = getFileSystem().open(path);
     assertEquals(0, instream.getPos());
     //expect initial read to fail
-    int result = instream.read();
-    assertMinusOne("initial byte read", result);
+    assertMinusOne("initial byte read", instream.read());
   }
 
   @Test
@@ -163,4 +170,126 @@ public abstract class AbstractContractOpenTest
     instream.close();
   }
 
+  @Test
+  public void testOpenFileReadZeroByte() throws Throwable {
+    describe("create & read a 0 byte file through the builders");
+    Path path = path("zero.txt");
+    FileSystem fs = getFileSystem();
+    fs.createFile(path).overwrite(true).build().close();
+    try (FSDataInputStream is = fs.openFile(path)
+        .opt("fs.test.something", true)
+        .opt("fs.test.something2", 3)
+        .opt("fs.test.something3", "3")
+        .build().get()) {
+      assertMinusOne("initial byte read", is.read());
+    }
+  }
+
+  @Test
+  public void testOpenFileUnknownOption() throws Throwable {
+    describe("calling openFile fails when a 'must()' option is unknown");
+    FutureDataInputStreamBuilder builder =
+        getFileSystem().openFile(path("testOpenFileUnknownOption"))
+        .opt("fs.test.something", true)
+        .must("fs.test.something", true);
+    intercept(IllegalArgumentException.class,
+        () -> builder.build());
+  }
+
+  @Test
+  public void testOpenFileLazyFail() throws Throwable {
+    describe("openFile fails on a missing file in the get() and not before");
+    FutureDataInputStreamBuilder builder =
+        getFileSystem().openFile(path("testOpenFileLazyFail"))
+            .opt("fs.test.something", true);
+    interceptFuture(FileNotFoundException.class, "", builder.build());
+  }
+
+  @Test
+  public void testOpenFileFailExceptionally() throws Throwable {
+    describe("openFile missing file chains into exceptionally()");
+    FutureDataInputStreamBuilder builder =
+        getFileSystem().openFile(path("testOpenFileFailExceptionally"))
+            .opt("fs.test.something", true);
+    assertNull("exceptional uprating",
+        builder.build().exceptionally(ex -> null).get());
+  }
+
+  @Test
+  public void testAwaitFutureFailToFNFE() throws Throwable {
+    describe("Verify that FutureIOSupport.awaitFuture extracts IOExceptions");
+    FutureDataInputStreamBuilder builder =
+        getFileSystem().openFile(path("testAwaitFutureFailToFNFE"))
+            .opt("fs.test.something", true);
+    intercept(FileNotFoundException.class,
+        () -> FutureIOSupport.awaitFuture(builder.build()));
+  }
+
+  @Test
+  public void testAwaitFutureTimeoutFailToFNFE() throws Throwable {
+    describe("Verify that FutureIOSupport.awaitFuture with a timeout works");
+    FutureDataInputStreamBuilder builder =
+        getFileSystem().openFile(path("testAwaitFutureFailToFNFE"))
+            .opt("fs.test.something", true);
+    intercept(FileNotFoundException.class,
+        () -> FutureIOSupport.awaitFuture(builder.build(),
+            10, TimeUnit.DAYS));
+  }
+
+  @Test
+  public void testOpenFileExceptionallyTranslating() throws Throwable {
+    describe("openFile missing file chains into exceptionally()");
+    CompletableFuture<FSDataInputStream> f = getFileSystem()
+        .openFile(path("testOpenFileUnknownOption")).build();
+    interceptFuture(RuntimeException.class,
+        "exceptionally",
+        f.exceptionally(ex -> {
+          throw new RuntimeException("exceptionally", ex);
+        }));
+  }
+
+  @Test
+  public void testChainedFailureAwaitFuture() throws Throwable {
+    describe("await Future handles chained failures");
+    CompletableFuture<FSDataInputStream> f = getFileSystem()
+        .openFile(path("testOpenFileUnknownOption"))
+        .build();
+    intercept(RuntimeException.class,
+        "exceptionally",
+        () -> FutureIOSupport.awaitFuture(
+            f.exceptionally(ex -> {
+              throw new RuntimeException("exceptionally", ex);
+            })));
+  }
+
+  @Test
+  public void testOpenFileApplyRead() throws Throwable {
+    describe("use the apply sequence to read a whole file");
+    Path path = path("testOpenFileApplyRead");
+    FileSystem fs = getFileSystem();
+    int len = 4096;
+    createFile(fs, path, true,
+        dataset(len, 0x40, 0x80));
+    CompletableFuture<Long> readAllBytes = fs.openFile(path)
+        .build()
+        .thenApply(ContractTestUtils::readStream);
+    assertEquals("Wrong number of bytes read value",
+        len,
+        (long) readAllBytes.get());
+  }
+
+  @Test
+  public void testOpenFileApplyAsyncRead() throws Throwable {
+    describe("verify that async accept callbacks are evaluated");
+    Path path = path("testOpenFileApplyAsyncRead");
+    FileSystem fs = getFileSystem();
+    createFile(fs, path, true,
+        dataset(4, 0x40, 0x80));
+    CompletableFuture<FSDataInputStream> future = fs.openFile(path).build();
+    AtomicBoolean accepted = new AtomicBoolean(false);
+    future.thenAcceptAsync(i -> accepted.set(true)).get();
+    assertTrue("async accept operation not invoked",
+        accepted.get());
+  }
+
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractPathHandleTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractPathHandleTest.java
index 36cfa6c..17043dc 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractPathHandleTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractPathHandleTest.java
@@ -17,16 +17,19 @@
  */
 package org.apache.hadoop.fs.contract;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
 import java.util.stream.Collectors;
 
 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.InvalidPathHandleException;
 import org.apache.hadoop.fs.Options.HandleOpt;
 import org.apache.hadoop.fs.Path;
@@ -38,6 +41,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyRead;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
 
 import org.apache.hadoop.fs.RawPathHandle;
 import org.junit.Test;
@@ -249,4 +253,61 @@ public abstract class AbstractContractPathHandleTest
     // unreachable
     return null;
   }
+
+
+  @Test
+  public void testOpenFileApplyRead() throws Throwable {
+    describe("use the apply sequence to read a whole file");
+    CompletableFuture<Long> readAllBytes = getFileSystem()
+        .openFile(
+            getHandleOrSkip(
+                testFile(B1)))
+        .build()
+        .thenApply(ContractTestUtils::readStream);
+    assertEquals("Wrong number of bytes read value",
+        TEST_FILE_LEN,
+        (long) readAllBytes.get());
+  }
+
+  @Test
+  public void testOpenFileDelete() throws Throwable {
+    describe("use the apply sequence to read a whole file");
+    FileStatus testFile = testFile(B1);
+    PathHandle handle = getHandleOrSkip(testFile);
+    // delete that file
+    FileSystem fs = getFileSystem();
+    fs.delete(testFile.getPath(), false);
+    // now construct the builder.
+    // even if the open happens in the build operation,
+    // the failure must not surface until later.
+    CompletableFuture<FSDataInputStream> builder =
+        fs.openFile(handle)
+            .opt("fs.test.something", true)
+            .build();
+    IOException ioe = interceptFuture(IOException.class, "", builder);
+    if (!(ioe instanceof FileNotFoundException)
+        && !(ioe instanceof InvalidPathHandleException)) {
+      // support both FileNotFoundException
+      // and InvalidPathHandleException as different implementations
+      // support either -and with non-atomic open sequences, possibly
+      // both
+      throw ioe;
+    }
+  }
+
+  @Test
+  public void testOpenFileLazyFail() throws Throwable {
+    describe("openFile fails on a misssng file in the get() and not before");
+    FileStatus stat = testFile(B1);
+    CompletableFuture<Long> readAllBytes = getFileSystem()
+        .openFile(
+            getHandleOrSkip(
+                stat))
+        .build()
+        .thenApply(ContractTestUtils::readStream);
+    assertEquals("Wrong number of bytes read value",
+        TEST_FILE_LEN,
+        (long) readAllBytes.get());
+  }
+
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
index ba12048..b4db3a5 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
@@ -1483,6 +1483,37 @@ public class ContractTestUtils extends Assert {
   }
 
   /**
+   * Function which calls {@code InputStream.read()} and
+   * downgrades an IOE to a runtime exception.
+   * @param in input
+   * @return the read value
+   * @throws AssertionError on any IOException
+   */
+  public static int read(InputStream in) {
+    try {
+      return in.read();
+    } catch (IOException ex) {
+      throw new AssertionError(ex);
+    }
+  }
+
+  /**
+   * Read a whole stream; downgrades an IOE to a runtime exception.
+   * @param in input
+   * @return the number of bytes read.
+   * @throws AssertionError on any IOException
+   */
+  public static long readStream(InputStream in) {
+    long count = 0;
+
+    while (read(in) >= 0) {
+      count++;
+    }
+    return count;
+  }
+
+
+  /**
    * Results of recursive directory creation/scan operations.
    */
   public static final class TreeScanResults {
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
index 67df6da..c1b6cc4 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java
@@ -30,6 +30,10 @@ import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Optional;
 import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 /**
@@ -691,6 +695,132 @@ public final class LambdaTestUtils {
   }
 
   /**
+   * Expect a future to raise a specific exception class when evaluated,
+   * <i>looking inside the raised {@code ExecutionException}</i> for it.
+   * @param clazz class of exception; the nested exception must be this class
+   * <i>or a subclass</i>.
+   *
+   * This is simply an unwrapping of the outcome of the future.
+   *
+   * If an exception is not raised, the return value of the {@code get()}
+   * call is included in the exception string.
+   *
+   * If the nested cause of the raised ExecutionException is not an
+   * Exception (i.e its an error), then the outer ExecutionException is
+   * rethrown.
+   * This keeps the operation signatures in sync.
+   *
+   * @param contained string which must be in the {@code toString()} value
+   * of the exception
+   * @param future future to get
+   * @param <T> return type of expression
+   * @param <E> exception class
+   * @return the caught exception if it was of the expected type and contents
+   * @throws AssertionError if the evaluation call didn't raise an exception.
+   * The error includes the {@code toString()} value of the result, if this
+   * can be determined.
+   * @throws CancellationException if the computation was cancelled
+   * @throws ExecutionException if the raised exception didn't contain an
+   * exception.
+   * @throws InterruptedException if the current thread was interrupted
+   * @throws TimeoutException if the wait timed out
+   * @throws Exception if the wrong exception was raised, or there was
+   * a text mismatch.
+   */
+  public static <T, E extends Throwable> E interceptFuture(
+      Class<E> clazz,
+      String contained,
+      Future<T> future) throws Exception {
+    return intercept(clazz,
+        contained,
+        () -> {
+          try {
+            return future.get();
+          } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause instanceof Exception) {
+              throw (Exception) cause;
+            } else {
+              throw e;
+            }
+          }
+        });
+  }
+
+  /**
+   * Expect a future to raise a specific exception class when evaluated,
+   * <i>looking inside the raised {@code ExecutionException}</i> for it.
+   * @param clazz class of exception; the nested exception must be this class
+   * <i>or a subclass</i>.
+   *
+   * This is simply an unwrapping of the outcome of the future.
+   *
+   * If an exception is not raised, the return value of the {@code get()}
+   * call is included in the exception string.
+   *
+   * If the nested cause of the raised ExecutionException is not an
+   * Exception (i.e its an error), then the outer ExecutionException is
+   * rethrown.
+   * This keeps the operation signatures in sync.
+   *
+   * @param contained string which must be in the {@code toString()} value
+   * of the exception
+   * @param future future to get
+   * @param <T> return type of expression
+   * @param <E> exception class
+   * @return the caught exception if it was of the expected type and contents
+   * @throws AssertionError if the evaluation call didn't raise an exception.
+   * The error includes the {@code toString()} value of the result, if this
+   * can be determined.
+   * @throws CancellationException if the computation was cancelled
+   * @throws ExecutionException if the raised exception didn't contain an
+   * exception.
+   * @throws InterruptedException if the current thread was interrupted
+   * @throws TimeoutException if the wait timed out
+   * @throws Exception if the wrong exception was raised, or there was
+   * a text mismatch.
+   */
+  public static <T, E extends Throwable> E interceptFuture(
+      final Class<E> clazz,
+      final String contained,
+      final long timeout,
+      final TimeUnit tu,
+      final Future<T> future) throws Exception {
+    return intercept(clazz,
+        contained,
+        () -> {
+          try {
+            return future.get(timeout, tu);
+          } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause instanceof Exception) {
+              throw (Exception) cause;
+            } else {
+              throw e;
+            }
+          }
+        });
+   }
+
+  /**
+   * Verify that the cause of an exception is of the given type.
+   * @param <E> exception class
+   * @param caught caught exception
+   * @return the extracted exception if it is of the expect type.
+   * @throws Exception the outer exception if there is no inner/wrong type
+   */
+  public static <E extends Throwable> E verifyCause(
+      Class<E> clazz,
+      final Throwable caught) throws Throwable {
+    Throwable cause = caught.getCause();
+    if (cause == null || !clazz.isAssignableFrom(cause.getClass())) {
+      throw caught;
+    } else {
+      return (E) caught;
+    }
+  }
+
+  /**
    * Returns {@code TimeoutException} on a timeout. If
    * there was a inner class passed in, includes it as the
    * inner failure.
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java
index 694fe73..479dd35 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestLambdaTestUtils.java
@@ -24,6 +24,10 @@ import org.junit.Test;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -516,17 +520,105 @@ public class TestLambdaTestUtils extends Assert {
    */
   @Test
   public void testEvalDoesWrapIOEs() throws Throwable {
-    AssertionError ex = intercept(AssertionError.class, "ioe",
-        () -> eval(() -> {
-          throw new IOException("ioe");
-        }));
-    Throwable cause = ex.getCause();
-    if (cause == null) {
-      throw ex;
-    }
-    if (!(cause instanceof IOException)) {
-      throw cause;
-    }
+    verifyCause(IOException.class,
+        intercept(AssertionError.class, "ioe",
+          () -> eval(() -> {
+            throw new IOException("ioe");
+          })));
+  }
+
+  @Test
+  public void testInterceptFutureUnwrapped() throws Throwable {
+    CompletableFuture<String> future = new CompletableFuture<>();
+    future.completeExceptionally(new IOException("oops"));
+    interceptFuture(IOException.class, "oops", future);
+  }
+
+  @Test
+  public void testInterceptFutureWrongException() throws Throwable {
+    CompletableFuture<String> future = new CompletableFuture<>();
+    future.completeExceptionally(new RuntimeException("oops"));
+    intercept(RuntimeException.class,
+        "oops",
+        () -> interceptFuture(IOException.class, "", future));
+  }
+
+  @Test
+  public void testInterceptFutureNotAnException() throws Throwable {
+    CompletableFuture<String> future = new CompletableFuture<>();
+    future.completeExceptionally(new Error("oops"));
+    verifyCause(Error.class,
+        intercept(ExecutionException.class,
+            "oops",
+            () -> interceptFuture(IOException.class, "", future)));
+  }
+
+  /**
+   * Variant for exception catching.
+   */
+  @Test
+  public void testInterceptFutureNotAnException2() throws Throwable {
+    CompletableFuture<String> future = new CompletableFuture<>();
+    future.completeExceptionally(new Error("oops"));
+    verifyCause(Error.class,
+        interceptFuture(ExecutionException.class, "", future));
+  }
+
+  @Test
+  public void testInterceptFutureNoFailures() throws Throwable {
+    CompletableFuture<String> future = new CompletableFuture<>();
+    future.complete("happy");
+    intercept(AssertionError.class,
+        "happy",
+        () -> interceptFuture(IOException.class, "oops", future));
+  }
+
+  /**
+   * This will timeout immediately and raise a TimeoutException.
+   */
+  @Test
+  public void testInterceptFutureTimeout() throws Throwable {
+    CompletableFuture<String> future = new CompletableFuture<>();
+    intercept(TimeoutException.class,
+        "",
+        () -> interceptFuture(IOException.class, "oops",
+            1, TimeUnit.NANOSECONDS,
+            future));
+  }
+
+  /**
+   * This will timeout immediately and raise a TimeoutException.
+   */
+  @Test
+  public void testInterceptFutureTimeout2() throws Throwable {
+    CompletableFuture<String> future = new CompletableFuture<>();
+    interceptFuture(TimeoutException.class, "",
+            1, TimeUnit.NANOSECONDS,
+            future);
+  }
+
+  /**
+   * This will timeout immediately and raise a TimeoutException.
+   */
+  @Test
+  public void testInterceptFutureTimeoutSuccess() throws Throwable {
+    CompletableFuture<String> future = new CompletableFuture<>();
+    future.completeExceptionally(new IOException("oops"));
+    interceptFuture(IOException.class, "oops",
+        1, TimeUnit.NANOSECONDS,
+        future);
+  }
+
+  /**
+   * This will timeout immediately and raise a TimeoutException.
+   */
+  @Test
+  public void testInterceptFutureCancelled() throws Throwable {
+    CompletableFuture<String> future = new CompletableFuture<>();
+    future.cancel(false);
+    interceptFuture(CancellationException.class, "",
+        1, TimeUnit.NANOSECONDS,
+        future);
   }
 
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 48d50e5..ed37f1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -3195,7 +3195,7 @@ public class DistributedFileSystem extends FileSystem
     }
 
     @Override
-    protected HdfsDataOutputStreamBuilder getThisBuilder() {
+    public HdfsDataOutputStreamBuilder getThisBuilder() {
       return this;
     }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractOpen.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractOpen.java
index 125e8ee..0d9e810 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractOpen.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractOpen.java
@@ -27,7 +27,7 @@ import org.junit.BeforeClass;
 import java.io.IOException;
 
 /**
- * Test dir operations on a the local FS.
+ * Test Open operations on HDFS.
  */
 public class TestHDFSContractOpen extends AbstractContractOpenTest {
 
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 bfc6c0e..1fcb118 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
@@ -25,9 +25,10 @@ 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.FileSystem;
+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;
@@ -36,6 +37,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.io.compress.Decompressor;
 import org.apache.hadoop.io.compress.SplitCompressionInputStream;
 import org.apache.hadoop.io.compress.SplittableCompressionCodec;
+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;
@@ -105,8 +107,12 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
     codec = compressionCodecs.getCodec(file);
 
     // open the file and seek to the start of the split
-    final FileSystem fs = file.getFileSystem(job);
-    fileIn = fs.open(file);
+    final FutureDataInputStreamBuilder builder =
+        file.getFileSystem(job).openFile(file);
+    FutureIOSupport.propagateOptions(builder, job,
+        MRJobConfig.INPUT_FILE_OPTION_PREFIX,
+        MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
+    fileIn = FutureIOSupport.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/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index b36b5ce..fb34698 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -1226,4 +1226,18 @@ public interface MRJobConfig {
       MR_AM_STAGING_DIR + ".erasurecoding.enabled";
 
   boolean DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED = false;
+
+  /**
+   * Prefix for options which are passed in to the filesystem
+   * after converting the subsequent dotted element to the schema.
+   */
+  @Unstable
+  String INPUT_FILE_OPTION_PREFIX = "mapreduce.job.input.file.option.";
+
+  /**
+   * Prefix for mandatory options which are passed in to the filesystem
+   * after converting the subsequent dotted element to the schema.
+   */
+  @Unstable
+  String INPUT_FILE_MANDATORY_PREFIX = "mapreduce.job.input.file.must.";
 }
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 71b2b79..c0ae9a5 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
@@ -25,9 +25,10 @@ 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.FileSystem;
+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;
@@ -36,6 +37,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.io.compress.CompressionInputStream;
 import org.apache.hadoop.io.compress.Decompressor;
 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.slf4j.Logger;
@@ -89,9 +91,13 @@ public class FixedLengthRecordReader
       numBytesToSkip = recordLength - partialRecordLength;
     }
 
-    // open the file and seek to the start of the split
-    final FileSystem fs = file.getFileSystem(job);
-    fileIn = fs.open(file);
+    // open the file
+    final FutureDataInputStreamBuilder builder =
+        file.getFileSystem(job).openFile(file);
+    FutureIOSupport.propagateOptions(builder, job,
+        MRJobConfig.INPUT_FILE_OPTION_PREFIX,
+        MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
+    fileIn = FutureIOSupport.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 ca85982..160c763 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
@@ -24,9 +24,10 @@ 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.FileSystem;
+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;
@@ -36,6 +37,7 @@ import org.apache.hadoop.io.compress.SplittableCompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.io.compress.Decompressor;
 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.slf4j.Logger;
@@ -82,8 +84,12 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
     final Path file = split.getPath();
 
     // open the file and seek to the start of the split
-    final FileSystem fs = file.getFileSystem(job);
-    fileIn = fs.open(file);
+    final FutureDataInputStreamBuilder builder =
+        file.getFileSystem(job).openFile(file);
+    FutureIOSupport.propagateOptions(builder, job,
+        MRJobConfig.INPUT_FILE_OPTION_PREFIX,
+        MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
+    fileIn = FutureIOSupport.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 7589961..dfff9ad 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
@@ -27,13 +27,15 @@ 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.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;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.LineReader;
@@ -93,10 +95,14 @@ public class NLineInputFormat extends FileInputFormat<LongWritable, Text> {
     if (status.isDirectory()) {
       throw new IOException("Not a file: " + fileName);
     }
-    FileSystem  fs = fileName.getFileSystem(conf);
     LineReader lr = null;
     try {
-      FSDataInputStream in  = fs.open(fileName);
+      final FutureDataInputStreamBuilder builder =
+          fileName.getFileSystem(conf).openFile(fileName);
+      FutureIOSupport.propagateOptions(builder, conf,
+          MRJobConfig.INPUT_FILE_OPTION_PREFIX,
+          MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
+      FSDataInputStream in  = FutureIOSupport.awaitFuture(builder.build());
       lr = new LineReader(in, conf);
       Text line = new Text();
       int numLines = 0;
diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
index 855aac9..bb6808f 100644
--- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
@@ -63,5 +63,10 @@
     <Method name="reopen"/>
     <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.fs.s3a.S3AFileSystem"/>
+    <Method name="openFileWithOptions"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
+  </Match>
 
 </FindBugsFilter>
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InternalConstants.java
new file mode 100644
index 0000000..bcf894f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InternalConstants.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Constants for internal use in the org.apache.hadoop.fs.s3a module itself.
+ * Please don't refer to these outside of this module & its tests.
+ * If you find you need to then either the code is doing something it
+ * should not, or these constants need to be uprated to being
+ * public and stable entries.
+ */
+@InterfaceAudience.Private
+public final class InternalConstants {
+
+  private InternalConstants() {
+  }
+
+  /**
+   * The known keys used in a standard openFile call.
+   * if there's a select marker in there then the keyset
+   * 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)));
+}
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 eb055dc..031a80b 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
@@ -39,6 +39,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -87,6 +88,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.s3a.select.InternalSelectConstants;
+import org.apache.hadoop.util.LambdaUtils;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -111,6 +114,8 @@ import org.apache.hadoop.fs.s3a.auth.delegation.AbstractS3ATokenIdentifier;
 import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.s3a.commit.PutTracker;
 import org.apache.hadoop.fs.s3a.commit.MagicCommitIntegration;
+import org.apache.hadoop.fs.s3a.select.SelectBinding;
+import org.apache.hadoop.fs.s3a.select.SelectConstants;
 import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
 import org.apache.hadoop.fs.s3a.s3guard.MetadataStoreListFilesIterator;
 import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
@@ -126,6 +131,7 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 
+import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Invoker.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
@@ -168,6 +174,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * retryable results in files being deleted.
   */
   public static final boolean DELETE_CONSIDERED_IDEMPOTENT = true;
+
   private URI uri;
   private Path workingDir;
   private String username;
@@ -224,6 +231,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   private S3ADataBlocks.BlockFactory blockFactory;
   private int blockOutputActiveBlocks;
   private WriteOperationHelper writeHelper;
+  private SelectBinding selectBinding;
   private boolean useListV1;
   private MagicCommitIntegration committerIntegration;
 
@@ -361,6 +369,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       committerIntegration = new MagicCommitIntegration(
           this, magicCommitterEnabled);
 
+      // instantiate S3 Select support
+      selectBinding = new SelectBinding(writeHelper);
+
       boolean blockUploadEnabled = conf.getBoolean(FAST_UPLOAD, true);
 
       if (!blockUploadEnabled) {
@@ -830,31 +841,87 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @param f the file name to open
    * @param bufferSize the size of the buffer to be used.
    */
+  @Retries.RetryTranslated
   public FSDataInputStream open(Path f, int bufferSize)
       throws IOException {
+    return open(f, Optional.empty());
+  }
+
+  /**
+   * Opens an FSDataInputStream at the indicated Path.
+   * @param path the file to open
+   * @param options configuration options if opened with the builder API.
+   * @throws IOException IO failure.
+   */
+  @Retries.RetryTranslated
+  private FSDataInputStream open(
+      final Path path,
+      final Optional<Configuration> options)
+      throws IOException {
+
     entryPoint(INVOCATION_OPEN);
-    LOG.debug("Opening '{}' for reading; input policy = {}", f, inputPolicy);
-    final FileStatus fileStatus = getFileStatus(f);
+    final FileStatus fileStatus = getFileStatus(path);
     if (fileStatus.isDirectory()) {
-      throw new FileNotFoundException("Can't open " + f
+      throw new FileNotFoundException("Can't open " + path
           + " because it is a directory");
     }
 
+    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);
+      readContext = createReadContext(fileStatus, policy, readAheadRange2);
+    } else {
+      readContext = createReadContext(fileStatus, inputPolicy, readAhead);
+    }
+    LOG.debug("Opening '{}'", readContext);
+
     return new FSDataInputStream(
-        new S3AInputStream(new S3AReadOpContext(hasMetadataStore(),
-            invoker,
-            s3guardInvoker,
-            statistics,
-            instrumentation,
-            fileStatus),
-            new S3ObjectAttributes(bucket,
-                pathToKey(f),
-                getServerSideEncryptionAlgorithm(),
-                encryptionSecrets.getEncryptionKey()),
+        new S3AInputStream(
+            readContext,
+            createObjectAttributes(path),
             fileStatus.getLen(),
-            s3,
-            readAhead,
-            inputPolicy));
+            s3));
+  }
+
+  /**
+   * 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 readAheadRange readahead value.
+   * @return a context for read and select operations.
+   */
+  private S3AReadOpContext createReadContext(
+      final FileStatus fileStatus,
+      final S3AInputPolicy seekPolicy,
+      final long readAheadRange) {
+    return new S3AReadOpContext(fileStatus.getPath(),
+        hasMetadataStore(),
+        invoker,
+        s3guardInvoker,
+        statistics,
+        instrumentation,
+        fileStatus,
+        seekPolicy,
+        readAheadRange);
+  }
+
+  /**
+   * Create the attributes of an object for a get/select request.
+   * @param f path path of the request.
+   * @return attributes to use when building the query.
+   */
+  private S3ObjectAttributes createObjectAttributes(final Path f) {
+    return new S3ObjectAttributes(bucket,
+        pathToKey(f),
+        getServerSideEncryptionAlgorithm(),
+        encryptionSecrets.getEncryptionKey());
   }
 
   /**
@@ -3549,6 +3616,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       // capability depends on FS configuration
       return isMagicCommitEnabled();
 
+    case SelectConstants.S3_SELECT_CAPABILITY:
+      // select is only supported if enabled
+      return selectBinding.isEnabled();
+
     default:
       return false;
     }
@@ -3576,4 +3647,104 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   protected void setTtlTimeProvider(S3Guard.ITtlTimeProvider ttlTimeProvider) {
     this.ttlTimeProvider = ttlTimeProvider;
   }
+
+  /**
+   * This is a proof of concept of a select API.
+   * Once a proper factory mechanism for opening files is added to the
+   * FileSystem APIs, this will be deleted <i>without any warning</i>.
+   * @param source path to source data
+   * @param expression select expression
+   * @param options request configuration from the builder.
+   * @return the stream of the results
+   * @throws IOException IO failure
+   */
+  @Retries.RetryTranslated
+  private FSDataInputStream select(final Path source,
+      final String expression,
+      final Configuration options)
+      throws IOException {
+    entryPoint(OBJECT_SELECT_REQUESTS);
+    requireSelectSupport(source);
+    final Path path = makeQualified(source);
+    // call getFileStatus(), which will look at S3Guard first,
+    // so the operation will fail if it is not there or S3Guard believes it has
+    // been deleted.
+    // validation of the file status are delegated to the binding.
+    final FileStatus fileStatus = getFileStatus(path);
+
+    // readahead range can be dynamically set
+    long ra = options.getLong(READAHEAD_RANGE, readAhead);
+    // build and execute the request
+    return selectBinding.select(
+        createReadContext(fileStatus, inputPolicy, ra),
+        expression,
+        options,
+        generateSSECustomerKey(),
+        createObjectAttributes(path));
+  }
+
+  /**
+   * Verify the FS supports S3 Select.
+   * @param source source file.
+   * @throws UnsupportedOperationException if not.
+   */
+  private void requireSelectSupport(final Path source) throws
+      UnsupportedOperationException {
+    if (!selectBinding.isEnabled()) {
+      throw new UnsupportedOperationException(
+          SelectConstants.SELECT_UNSUPPORTED);
+    }
+  }
+
+  /**
+   * Initiate the open or select operation.
+   * This is invoked from both the FileSystem and FileContext APIs
+   * @param path path to the file
+   * @param mandatoryKeys set of options declared as mandatory.
+   * @param options options set during the build sequence.
+   * @return a future which will evaluate to the opened/selected file.
+   * @throws IOException failure to resolve the link.
+   * @throws PathIOException operation is a select request but S3 select is
+   * disabled
+   * @throws IllegalArgumentException unknown mandatory key
+   */
+  @Override
+  @Retries.RetryTranslated
+  public CompletableFuture<FSDataInputStream> openFileWithOptions(
+      final Path path,
+      final Set<String> mandatoryKeys,
+      final Configuration options,
+      final int bufferSize) throws IOException {
+    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");
+    }
+    CompletableFuture<FSDataInputStream> result = new CompletableFuture<>();
+    if (!isSelect) {
+      // normal path.
+      unboundedThreadPool.submit(() ->
+          LambdaUtils.eval(result,
+              () -> open(path, Optional.of(options))));
+    } else {
+      // it is a select statement.
+      // fail fast if the method is not present
+      requireSelectSupport(path);
+      // submit the query
+      unboundedThreadPool.submit(() ->
+          LambdaUtils.eval(result,
+              () -> select(path, sql, options)));
+    }
+    return result;
+  }
+
 }
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 68f98e4..ccc86d0 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
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import javax.annotation.Nullable;
+
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.model.GetObjectRequest;
 import com.amazonaws.services.s3.model.S3Object;
@@ -60,6 +62,10 @@ import static org.apache.commons.lang3.StringUtils.isNotEmpty;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class S3AInputStream extends FSInputStream implements CanSetReadahead {
+
+  public static final String E_NEGATIVE_READAHEAD_VALUE
+      = "Negative readahead value";
+
   /**
    * This is the public position; the one set in {@link #seek(long)}
    * and returned in {@link #getPos()}.
@@ -112,12 +118,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
    * @param s3Attributes object attributes from a HEAD request
    * @param contentLength length of content
    * @param client S3 client to use
-   * @param readahead readahead bytes
-   * @param inputPolicy IO policy
    */
-  public S3AInputStream(S3AReadOpContext ctx, S3ObjectAttributes s3Attributes,
-      long contentLength, AmazonS3 client, long readahead,
-      S3AInputPolicy inputPolicy) {
+  public S3AInputStream(S3AReadOpContext ctx,
+      S3ObjectAttributes s3Attributes,
+      long contentLength,
+      AmazonS3 client) {
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()),
         "No Bucket");
     Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key");
@@ -133,8 +138,8 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     this.serverSideEncryptionAlgorithm =
         s3Attributes.getServerSideEncryptionAlgorithm();
     this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey();
-    setInputPolicy(inputPolicy);
-    setReadahead(readahead);
+    setInputPolicy(ctx.getInputPolicy());
+    setReadahead(ctx.getReadahead());
   }
 
   /**
@@ -179,7 +184,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     }
     String text = String.format("Failed to %s %s at %d",
         (opencount == 0 ? "open" : "re-open"), uri, targetPos);
-    S3Object object = context.getReadInvoker().once(text, uri,
+    S3Object object = Invoker.once(text, uri,
         () -> client.getObject(request));
     wrappedStream = object.getObjectContent();
     contentRangeStart = targetPos;
@@ -722,12 +727,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
 
   @Override
   public synchronized void setReadahead(Long readahead) {
-    if (readahead == null) {
-      this.readahead = Constants.DEFAULT_READAHEAD_RANGE;
-    } else {
-      Preconditions.checkArgument(readahead >= 0, "Negative readahead value");
-      this.readahead = readahead;
-    }
+    this.readahead = validateReadahead(readahead);
   }
 
   /**
@@ -780,4 +780,19 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     return rangeLimit;
   }
 
+  /**
+   * from a possibly null Long value, return a valid
+   * readahead.
+   * @param readahead new readahead
+   * @return a natural number.
+   * @throws IllegalArgumentException if the range is invalid.
+   */
+  public static long validateReadahead(@Nullable Long readahead) {
+    if (readahead == null) {
+      return Constants.DEFAULT_READAHEAD_RANGE;
+    } else {
+      Preconditions.checkArgument(readahead >= 0, E_NEGATIVE_READAHEAD_VALUE);
+      return readahead;
+    }
+  }
 }
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 78ba47d..17c5aff 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
@@ -160,6 +160,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
       OBJECT_PUT_BYTES,
       OBJECT_PUT_REQUESTS,
       OBJECT_PUT_REQUESTS_COMPLETED,
+      OBJECT_SELECT_REQUESTS,
       STREAM_WRITE_FAILURES,
       STREAM_WRITE_BLOCK_UPLOADS,
       STREAM_WRITE_BLOCK_UPLOADS_COMMITTED,
@@ -550,7 +551,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
    * Create a stream input statistics instance.
    * @return the new instance
    */
-  InputStreamStatistics newInputStreamStatistics() {
+  public InputStreamStatistics newInputStreamStatistics() {
     return new InputStreamStatistics();
   }
 
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java
index fba39b9..553d02f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOpContext.java
@@ -84,4 +84,29 @@ public class S3AOpContext {
         dstFileStatus);
   }
 
+  public boolean isS3GuardEnabled() {
+    return isS3GuardEnabled;
+  }
+
+  public Invoker getInvoker() {
+    return invoker;
+  }
+
+  @Nullable
+  public FileSystem.Statistics getStats() {
+    return stats;
+  }
+
+  public S3AInstrumentation getInstrumentation() {
+    return instrumentation;
+  }
+
+  @Nullable
+  public Invoker getS3guardInvoker() {
+    return s3guardInvoker;
+  }
+
+  public FileStatus getDstFileStatus() {
+    return dstFileStatus;
+  }
 }
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 220cd0d..73c2194 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
@@ -20,29 +20,69 @@ package org.apache.hadoop.fs.s3a;
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 import javax.annotation.Nullable;
 
+import com.google.common.base.Preconditions;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * Read-specific operation context struct.
  */
 public class S3AReadOpContext extends S3AOpContext {
-  public S3AReadOpContext(boolean isS3GuardEnabled, Invoker invoker,
-      Invoker s3guardInvoker, @Nullable FileSystem.Statistics stats,
-      S3AInstrumentation instrumentation, FileStatus dstFileStatus) {
+
+  /**
+   * Path of read.
+   */
+  private final Path path;
+
+  /**
+   * Initial input policy of the stream.
+   */
+  private final S3AInputPolicy inputPolicy;
+
+  /**
+   * Readahead for GET operations/skip, etc.
+   */
+  private final long readahead;
+
+  /**
+   * Instantiate.
+   * @param path path of read
+   * @param isS3GuardEnabled true iff S3Guard is enabled.
+   * @param invoker invoker for normal retries.
+   * @param s3guardInvoker S3Guard-specific retry invoker.
+   * @param stats statistics (may be null)
+   * @param instrumentation FS instrumentation
+   * @param dstFileStatus target file status
+   * @param inputPolicy the input policy
+   * @param readahead readahead for GET operations/skip, etc.
+   */
+  public S3AReadOpContext(
+      final Path path,
+      boolean isS3GuardEnabled,
+      Invoker invoker,
+      Invoker s3guardInvoker,
+      @Nullable FileSystem.Statistics stats,
+      S3AInstrumentation instrumentation,
+      FileStatus dstFileStatus,
+      S3AInputPolicy inputPolicy,
+      final long readahead) {
     super(isS3GuardEnabled, invoker, s3guardInvoker, stats, instrumentation,
         dstFileStatus);
-  }
-
-  public S3AReadOpContext(boolean isS3GuardEnabled, Invoker invoker,
-      @Nullable FileSystem.Statistics stats, S3AInstrumentation instrumentation,
-      FileStatus dstFileStatus) {
-    super(isS3GuardEnabled, invoker, stats, instrumentation, dstFileStatus);
+    this.path = checkNotNull(path);
+    Preconditions.checkArgument(readahead >= 0,
+        "invalid readahead %d", readahead);
+    this.inputPolicy = checkNotNull(inputPolicy);
+    this.readahead = readahead;
   }
 
   /**
-   * Get invoker to use for read operations.  When S3Guard is enabled we use
-   * the S3Guard invoker, which deals with things like FileNotFoundException
+   * Get invoker to use for read operations.
+   * When S3Guard is enabled we use the S3Guard invoker,
+   * which deals with things like FileNotFoundException
    * differently.
    * @return invoker to use for read codepaths
    */
@@ -53,4 +93,39 @@ public class S3AReadOpContext extends S3AOpContext {
       return invoker;
     }
   }
+
+  /**
+   * Get the path of this read.
+   * @return path.
+   */
+  public Path getPath() {
+    return path;
+  }
+
+  /**
+   * Get the IO policy.
+   * @return the initial input policy.
+   */
+  public S3AInputPolicy getInputPolicy() {
+    return inputPolicy;
+  }
+
+  /**
+   * Get the readahead for this operation.
+   * @return a value >= 0
+   */
+  public long getReadahead() {
+    return readahead;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "S3AReadOpContext{");
+    sb.append("path=").append(path);
+    sb.append(", inputPolicy=").append(inputPolicy);
+    sb.append(", readahead=").append(readahead);
+    sb.append('}');
+    return sb.toString();
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 6182b43..8428831 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -83,6 +83,7 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 
+import static org.apache.commons.lang3.StringUtils.isEmpty;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 
 /**
@@ -250,6 +251,12 @@ public final class S3AUtils {
         ioe.initCause(ase);
         break;
 
+      // method not allowed; seen on S3 Select.
+      // treated as a bad request
+      case 405:
+        ioe = new AWSBadRequestException(message, s3Exception);
+        break;
+
       // out of range. This may happen if an object is overwritten with
       // a shorter one while it is being read.
       case 416:
@@ -943,7 +950,7 @@ public final class S3AUtils {
       String key,
       String val,
       String defVal) throws IOException {
-    return StringUtils.isEmpty(val)
+    return isEmpty(val)
         ? lookupPassword(conf, key, defVal)
         : val;
   }
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 d1bff8a..d67e3e1 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
@@ -18,19 +18,24 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
 /**
  * This class is only a holder for bucket, key, SSE Algorithm and SSE key
- * attributes. It is only used in {@link S3AInputStream}
+ * attributes. It is used in {@link S3AInputStream} and the select equivalent.
  * as a way to reduce parameters being passed
  * to the constructor of such class.
  */
-class S3ObjectAttributes {
-  private String bucket;
-  private String key;
-  private S3AEncryptionMethods serverSideEncryptionAlgorithm;
-  private String serverSideEncryptionKey;
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class S3ObjectAttributes {
+  private final String bucket;
+  private final String key;
+  private final S3AEncryptionMethods serverSideEncryptionAlgorithm;
+  private final String serverSideEncryptionKey;
 
-  S3ObjectAttributes(
+  public S3ObjectAttributes(
       String bucket,
       String key,
       S3AEncryptionMethods serverSideEncryptionAlgorithm,
@@ -41,19 +46,19 @@ class S3ObjectAttributes {
     this.serverSideEncryptionKey = serverSideEncryptionKey;
   }
 
-  String getBucket() {
+  public String getBucket() {
     return bucket;
   }
 
-  String getKey() {
+  public String getKey() {
     return key;
   }
 
-  S3AEncryptionMethods getServerSideEncryptionAlgorithm() {
+  public S3AEncryptionMethods getServerSideEncryptionAlgorithm() {
     return serverSideEncryptionAlgorithm;
   }
 
-  String getServerSideEncryptionKey() {
+  public String getServerSideEncryptionKey() {
     return serverSideEncryptionKey;
   }
 }
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 10ae1db..6f79286 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
@@ -102,6 +102,8 @@ public enum Statistic {
   OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"),
   OBJECT_PUT_BYTES_PENDING("object_put_bytes_pending",
       "number of bytes queued for upload/being actively uploaded"),
+  OBJECT_SELECT_REQUESTS("object_select_requests",
+      "Count of S3 Select requests issued"),
   STREAM_ABORTED("stream_aborted",
       "Count of times the TCP stream was aborted"),
   STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations",
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
index a5f6817..fcc16a1 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
 import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
 import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
@@ -34,6 +35,8 @@ import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.PartETag;
 import com.amazonaws.services.s3.model.PutObjectRequest;
 import com.amazonaws.services.s3.model.PutObjectResult;
+import com.amazonaws.services.s3.model.SelectObjectContentRequest;
+import com.amazonaws.services.s3.model.SelectObjectContentResult;
 import com.amazonaws.services.s3.model.UploadPartRequest;
 import com.amazonaws.services.s3.model.UploadPartResult;
 import com.amazonaws.services.s3.transfer.model.UploadResult;
@@ -45,17 +48,19 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.commit.DurationInfo;
+import org.apache.hadoop.fs.s3a.select.SelectBinding;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.hadoop.fs.s3a.Invoker.*;
 
 /**
- * Helper for low-level operations against an S3 Bucket for writing data
- * and creating and committing pending writes.
+ * Helper for low-level operations against an S3 Bucket for writing data,
+ * creating and committing pending writes, and other S3-layer operations.
  * <p>
  * It hides direct access to the S3 API
- * and is a location where the object upload process can be evolved/enhanced.
+ * and is a location where the object operations can be evolved/enhanced.
  * <p>
  * Features
  * <ul>
@@ -65,8 +70,10 @@ import static org.apache.hadoop.fs.s3a.Invoker.*;
  *   errors.</li>
  *   <li>Callbacks to let the FS know of events in the output stream
  *   upload process.</li>
+ *   <li>Other low-level access to S3 functions, for private use.</li>
  *   <li>Failure handling, including converting exceptions to IOEs.</li>
  *   <li>Integration with instrumentation and S3Guard.</li>
+ *   <li>Evolution to add more low-level operations, such as S3 select.</li>
  * </ul>
  *
  * This API is for internal use only.
@@ -76,9 +83,24 @@ import static org.apache.hadoop.fs.s3a.Invoker.*;
 public class WriteOperationHelper {
   private static final Logger LOG =
       LoggerFactory.getLogger(WriteOperationHelper.class);
+
+  /**
+   * Owning filesystem.
+   */
   private final S3AFileSystem owner;
+
+  /**
+   * Invoker for operations; uses the S3A retry policy and calls int
+   * {@link #operationRetried(String, Exception, int, boolean)} on retries.
+   */
   private final Invoker invoker;
 
+  /** Configuration of the owner. This is a reference, not a copy. */
+  private final Configuration conf;
+
+  /** Bucket of the owner FS. */
+  private final String bucket;
+
   /**
    * Constructor.
    * @param owner owner FS creating the helper
@@ -89,6 +111,8 @@ public class WriteOperationHelper {
     this.owner = owner;
     this.invoker = new Invoker(new S3ARetryPolicy(conf),
         this::operationRetried);
+    this.conf = conf;
+    bucket = owner.getBucket();
   }
 
   /**
@@ -189,7 +213,7 @@ public class WriteOperationHelper {
   public String initiateMultiPartUpload(String destKey) throws IOException {
     LOG.debug("Initiating Multipart upload to {}", destKey);
     final InitiateMultipartUploadRequest initiateMPURequest =
-        new InitiateMultipartUploadRequest(owner.getBucket(),
+        new InitiateMultipartUploadRequest(bucket,
             destKey,
             newObjectMetadata(-1));
     initiateMPURequest.setCannedACL(owner.getCannedACL());
@@ -231,7 +255,7 @@ public class WriteOperationHelper {
           // attempt to sort an unmodifiable list.
           CompleteMultipartUploadResult result =
               owner.getAmazonS3Client().completeMultipartUpload(
-                  new CompleteMultipartUploadRequest(owner.getBucket(),
+                  new CompleteMultipartUploadRequest(bucket,
                       destKey,
                       uploadId,
                       new ArrayList<>(partETags)));
@@ -381,7 +405,7 @@ public class WriteOperationHelper {
     LOG.debug("Creating part upload request for {} #{} size {}",
         uploadId, partNumber, size);
     UploadPartRequest request = new UploadPartRequest()
-        .withBucketName(owner.getBucket())
+        .withBucketName(bucket)
         .withKey(destKey)
         .withUploadId(uploadId)
         .withPartNumber(partNumber)
@@ -409,7 +433,7 @@ public class WriteOperationHelper {
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder(
-        "WriteOperationHelper {bucket=").append(owner.getBucket());
+        "WriteOperationHelper {bucket=").append(bucket);
     sb.append('}');
     return sb.toString();
   }
@@ -478,4 +502,71 @@ public class WriteOperationHelper {
         () -> owner.uploadPart(request));
   }
 
+  /**
+   * Get the configuration of this instance; essentially the owning
+   * filesystem configuration.
+   * @return the configuration.
+   */
+  public Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * Create a S3 Select request for the destination path.
+   * This does not build the query.
+   * @param path pre-qualified path for query
+   * @return the request
+   */
+  public SelectObjectContentRequest newSelectRequest(Path path) {
+    SelectObjectContentRequest request = new SelectObjectContentRequest();
+    request.setBucketName(bucket);
+    request.setKey(owner.pathToKey(path));
+    return request;
+  }
+
+  /**
+   * Execute an S3 Select operation.
+   * On a failure, the request is only logged at debug to avoid the
+   * select exception being printed.
+   * @param source source for selection
+   * @param request Select request to issue.
+   * @param action the action for use in exception creation
+   * @return response
+   * @throws IOException failure
+   */
+  @Retries.RetryTranslated
+  public SelectObjectContentResult select(
+      final Path source,
+      final SelectObjectContentRequest request,
+      final String action)
+      throws IOException {
+    String bucketName = request.getBucketName();
+    Preconditions.checkArgument(bucket.equals(bucketName),
+        "wrong bucket: %s", bucketName);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Initiating select call {} {}",
+          source, request.getExpression());
+      LOG.debug(SelectBinding.toString(request));
+    }
+    return invoker.retry(
+        action,
+        source.toString(),
+        true,
+        () -> {
+          try (DurationInfo ignored =
+                   new DurationInfo(LOG, "S3 Select operation")) {
+            try {
+              return owner.getAmazonS3Client().selectObjectContent(request);
+            } catch (AmazonS3Exception e) {
+              LOG.error("Failure of S3 Select request against {}",
+                  source);
+              LOG.debug("S3 Select request against {}:\n{}",
+                  source,
+                  SelectBinding.toString(request),
+                  e);
+              throw e;
+            }
+          }
+        });
+  }
 }
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 3751fda..e4fd064 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
@@ -53,6 +53,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens;
 import org.apache.hadoop.fs.s3a.commit.CommitConstants;
+import org.apache.hadoop.fs.s3a.select.SelectTool;
 import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ExitUtil;
@@ -90,14 +91,15 @@ public abstract class S3GuardTool extends Configured implements Tool {
       "\t" + Uploads.NAME + " - " + Uploads.PURPOSE + "\n" +
       "\t" + Diff.NAME + " - " + Diff.PURPOSE + "\n" +
       "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n" +
-      "\t" + SetCapacity.NAME + " - " +SetCapacity.PURPOSE + "\n";
+      "\t" + SetCapacity.NAME + " - " + SetCapacity.PURPOSE + "\n" +
+      "\t" + SelectTool.NAME + " - " + SelectTool.PURPOSE + "\n";
   private static final String DATA_IN_S3_IS_PRESERVED
       = "(all data in S3 is preserved)";
 
   public static final String E_NO_METASTORE_OR_FILESYSTEM
       = "No metastore or filesystem specified";
 
-  abstract public String getUsage();
+  public abstract String getUsage();
 
   // Exit codes
   static final int SUCCESS = EXIT_SUCCESS;
@@ -144,19 +146,19 @@ public abstract class S3GuardTool extends Configured implements Tool {
   /**
    * Return sub-command name.
    */
-  abstract String getName();
+  public abstract String getName();
 
   /**
    * Parse DynamoDB region from either -m option or a S3 path.
    *
-   * This function should only be called from {@link Init} or
-   * {@link Destroy}.
+   * This function should only be called from {@link S3GuardTool.Init} or
+   * {@link S3GuardTool.Destroy}.
    *
    * @param paths remaining parameters from CLI.
    * @throws IOException on I/O errors.
    * @throws ExitUtil.ExitException on validation errors
    */
-  void parseDynamoDBRegion(List<String> paths) throws IOException {
+  protected void parseDynamoDBRegion(List<String> paths) throws IOException {
     Configuration conf = getConf();
     String fromCli = getCommandFormat().getOptValue(REGION_FLAG);
     String fromConf = conf.get(S3GUARD_DDB_REGION_KEY);
@@ -269,7 +271,8 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * @param forceCreate override the auto-creation setting to true.
    * @return a initialized metadata store.
    */
-  MetadataStore initMetadataStore(boolean forceCreate) throws IOException {
+  protected MetadataStore initMetadataStore(boolean forceCreate)
+      throws IOException {
     if (getStore() != null) {
       return getStore();
     }
@@ -334,7 +337,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * @throws IOException failure to init filesystem
    * @throws ExitUtil.ExitException if the FS is not an S3A FS
    */
-  void initS3AFileSystem(String path) throws IOException {
+  protected void initS3AFileSystem(String path) throws IOException {
     URI uri = toUri(path);
     // Make sure that S3AFileSystem does not hold an actual MetadataStore
     // implementation.
@@ -367,7 +370,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * @param args command line arguments.
    * @return the position arguments from CLI.
    */
-  List<String> parseArgs(String[] args) {
+  protected List<String> parseArgs(String[] args) {
     return getCommandFormat().parse(args, 1);
   }
 
@@ -404,16 +407,16 @@ public abstract class S3GuardTool extends Configured implements Tool {
    *
    * As well as returning an exit code, the implementations can choose to
    * throw an instance of {@link ExitUtil.ExitException} with their exit
-   * code set to the desired exit value. The exit code of auch an exception
+   * code set to the desired exit value. The exit code of such an exception
    * is used for the tool's exit code, and the stack trace only logged at
    * debug.
    * @param args argument list
    * @param out output stream
    * @return the exit code to return.
    * @throws Exception on any failure
-   * @throws ExitUtil.ExitException for an alternative clean exit
    */
-  public abstract int run(String[] args, PrintStream out) throws Exception;
+  public abstract int run(String[] args, PrintStream out) throws Exception,
+      ExitUtil.ExitException;
 
   /**
    * Create the metadata store.
@@ -448,7 +451,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    String getName() {
+    public String getName() {
       return NAME;
     }
 
@@ -541,7 +544,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    String getName() {
+    public String getName() {
       return NAME;
     }
 
@@ -613,7 +616,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    String getName() {
+    public String getName() {
       return NAME;
     }
 
@@ -678,7 +681,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    String getName() {
+    public String getName() {
       return NAME;
     }
 
@@ -810,7 +813,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    String getName() {
+    public String getName() {
       return NAME;
     }
 
@@ -1018,7 +1021,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    String getName() {
+    public String getName() {
       return NAME;
     }
 
@@ -1108,7 +1111,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    String getName() {
+    public String getName() {
       return NAME;
     }
 
@@ -1290,7 +1293,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    String getName() {
+    public String getName() {
       return NAME;
     }
 
@@ -1457,7 +1460,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     return uri;
   }
 
-  private static void printHelp(S3GuardTool tool) {
+  protected static void printHelp(S3GuardTool tool) {
     if (tool == null) {
       errorln("Usage: hadoop " + USAGE);
       errorln("\tperform S3Guard metadata store " +
@@ -1469,11 +1472,11 @@ public abstract class S3GuardTool extends Configured implements Tool {
     errorln(COMMON_USAGE);
   }
 
-  private static void errorln() {
+  protected static void errorln() {
     System.err.println();
   }
 
-  private static void errorln(String x) {
+  protected static void errorln(String x) {
     System.err.println(x);
   }
 
@@ -1483,7 +1486,9 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * @param format format string
    * @param args optional arguments
    */
-  private static void println(PrintStream out, String format, Object... args) {
+  protected static void println(PrintStream out,
+      String format,
+      Object... args) {
     out.println(String.format(format, args));
   }
 
@@ -1523,8 +1528,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
    */
   protected static ExitUtil.ExitException invalidArgs(
       String format, Object...args) {
-    return new ExitUtil.ExitException(INVALID_ARGUMENT,
-        String.format(format, args));
+    return exitException(INVALID_ARGUMENT, format, args);
   }
 
   /**
@@ -1535,8 +1539,8 @@ public abstract class S3GuardTool extends Configured implements Tool {
    */
   protected static ExitUtil.ExitException badState(
       String format, Object...args) {
-    return new ExitUtil.ExitException(E_BAD_STATE,
-        String.format(format, args));
+    int exitCode = E_BAD_STATE;
+    return exitException(exitCode, format, args);
   }
 
   /**
@@ -1547,7 +1551,22 @@ public abstract class S3GuardTool extends Configured implements Tool {
    */
   protected static ExitUtil.ExitException userAborted(
       String format, Object...args) {
-    return new ExitUtil.ExitException(ERROR, String.format(format, args));
+    return exitException(ERROR, format, args);
+  }
+
+  /**
+   * Build a exception to throw with a formatted message.
+   * @param exitCode exit code to use
+   * @param format string format
+   * @param args optional arguments for the string
+   * @return a new exception to throw
+   */
+  protected static ExitUtil.ExitException exitException(
+      final int exitCode,
+      final String format,
+      final Object... args) {
+    return new ExitUtil.ExitException(exitCode,
+        String.format(format, args));
   }
 
 
@@ -1607,6 +1626,11 @@ public abstract class S3GuardTool extends Configured implements Tool {
     case Uploads.NAME:
       command = new Uploads(conf);
       break;
+    case SelectTool.NAME:
+      // the select tool is not technically a S3Guard tool, but it's on the CLI
+      // because this is the defacto S3 CLI.
+      command = new SelectTool(conf);
+      break;
     default:
       printHelp(null);
       throw new ExitUtil.ExitException(E_USAGE,
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
new file mode 100644
index 0000000..ae3dc08
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java
@@ -0,0 +1,77 @@
+/*
+ * 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.select;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.s3a.InternalConstants;
+
+import static org.apache.hadoop.fs.s3a.select.SelectConstants.*;
+
+/**
+ * Constants for internal use in the org.apache.hadoop.fs.s3a module itself.
+ * Please don't refer to these outside of this module & its tests.
+ * If you find you need to then either the code is doing something it
+ * should not, or these constants need to be uprated to being
+ * public and stable entries.
+ */
+@InterfaceAudience.Private
+public final class InternalSelectConstants {
+
+  private InternalSelectConstants() {
+  }
+
+  /**
+   * An unmodifiable set listing the options
+   * supported in {@code openFile()}.
+   */
+  public static final Set<String> SELECT_OPTIONS;
+
+  /*
+   * Build up the options, pulling in the standard set too.
+   */
+  static {
+    // when adding to this, please keep in alphabetical order after the
+    // common options and the SQL.
+    HashSet<String> options = new HashSet<>(Arrays.asList(
+        SELECT_SQL,
+        SELECT_ERRORS_INCLUDE_SQL,
+        SELECT_INPUT_COMPRESSION,
+        SELECT_INPUT_FORMAT,
+        SELECT_OUTPUT_FORMAT,
+        CSV_INPUT_COMMENT_MARKER,
+        CSV_INPUT_HEADER,
+        CSV_INPUT_INPUT_FIELD_DELIMITER,
+        CSV_INPUT_QUOTE_CHARACTER,
+        CSV_INPUT_QUOTE_ESCAPE_CHARACTER,
+        CSV_INPUT_RECORD_DELIMITER,
+        CSV_OUTPUT_FIELD_DELIMITER,
+        CSV_OUTPUT_QUOTE_CHARACTER,
+        CSV_OUTPUT_QUOTE_ESCAPE_CHARACTER,
+        CSV_OUTPUT_QUOTE_FIELDS,
+        CSV_OUTPUT_RECORD_DELIMITER
+    ));
+    options.addAll(InternalConstants.STANDARD_OPENFILE_KEYS);
+    SELECT_OPTIONS = Collections.unmodifiableSet(options);
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java
new file mode 100644
index 0000000..ff39b9a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java
@@ -0,0 +1,431 @@
+/*
+ * 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.select;
+
+import java.io.IOException;
+import java.util.Locale;
+import java.util.Optional;
+
+import com.amazonaws.services.s3.model.CSVInput;
+import com.amazonaws.services.s3.model.CSVOutput;
+import com.amazonaws.services.s3.model.ExpressionType;
+import com.amazonaws.services.s3.model.InputSerialization;
+import com.amazonaws.services.s3.model.OutputSerialization;
+import com.amazonaws.services.s3.model.QuoteFields;
+import com.amazonaws.services.s3.model.SSECustomerKey;
+import com.amazonaws.services.s3.model.SelectObjectContentRequest;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.apache.hadoop.fs.s3a.S3AReadOpContext;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.fs.s3a.WriteOperationHelper;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.commons.lang3.StringUtils.isNotEmpty;
+import static org.apache.hadoop.fs.s3a.select.SelectConstants.*;
+
+/**
+ * Class to do the S3 select binding and build a select request from the
+ * supplied arguments/configuration.
+ *
+ * This class is intended to be instantiated by the owning S3AFileSystem
+ * instance to handle the construction of requests: IO is still done exclusively
+ * in the filesystem.
+ */
+public class SelectBinding {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(SelectBinding.class);
+
+  /** Operations on the store. */
+  private final WriteOperationHelper operations;
+
+  /** Is S3 Select enabled? */
+  private final boolean enabled;
+  private final boolean errorsIncludeSql;
+
+  /**
+   * Constructor.
+   * @param operations owning FS.
+   */
+  public SelectBinding(final WriteOperationHelper operations) {
+    this.operations = checkNotNull(operations);
+    Configuration conf = getConf();
+    this.enabled = conf.getBoolean(FS_S3A_SELECT_ENABLED, true);
+    this.errorsIncludeSql = conf.getBoolean(SELECT_ERRORS_INCLUDE_SQL, false);
+  }
+
+  Configuration getConf() {
+    return operations.getConf();
+  }
+
+  /**
+   * Is the service supported?
+   * @return true iff select is enabled.
+   */
+  public boolean isEnabled() {
+    return enabled;
+  }
+
+  /**
+   * Build and execute a select request.
+   * @param readContext the read context, which includes the source path.
+   * @param expression the SQL expression.
+   * @param builderOptions query options
+   * @param sseKey optional SSE customer key
+   * @param objectAttributes object attributes from a HEAD request
+   * @return an FSDataInputStream whose wrapped stream is a SelectInputStream
+   * @throws IllegalArgumentException argument failure
+   * @throws IOException failure building, validating or executing the request.
+   * @throws PathIOException source path is a directory.
+   */
+  @Retries.RetryTranslated
+  public FSDataInputStream select(
+      final S3AReadOpContext readContext,
+      final String expression,
+      final Configuration builderOptions,
+      final Optional<SSECustomerKey> sseKey,
+      final S3ObjectAttributes objectAttributes) throws IOException {
+
+    return new FSDataInputStream(
+        executeSelect(readContext,
+            objectAttributes,
+            builderOptions,
+            buildSelectRequest(
+                readContext.getPath(),
+                expression,
+                builderOptions,
+                sseKey)));
+  }
+
+  /**
+   * Build a select request.
+   * @param path source path.
+   * @param expression the SQL expression.
+   * @param builderOptions config to extract other query options from
+   * @param sseKey optional SSE customer key
+   * @return the request to serve
+   * @throws IllegalArgumentException argument failure
+   * @throws IOException problem building/validating the request
+   */
+  public SelectObjectContentRequest buildSelectRequest(
+      final Path path,
+      final String expression,
+      final Configuration builderOptions,
+      final Optional<SSECustomerKey> sseKey)
+      throws IOException {
+    Preconditions.checkState(isEnabled(),
+        "S3 Select is not enabled for %s", path);
+
+    SelectObjectContentRequest request = operations.newSelectRequest(path);
+    buildRequest(request, expression, builderOptions);
+    // optionally set an SSE key in the input
+    sseKey.ifPresent(request::withSSECustomerKey);
+    return request;
+  }
+
+  /**
+   * Execute the select request.
+   * @param readContext read context
+   * @param objectAttributes object attributes from a HEAD request
+   * @param builderOptions the options which came in from the openFile builder.
+   * @param request the built up select request.
+   * @return a SelectInputStream
+   * @throws IOException failure
+   * @throws PathIOException source path is a directory.
+   */
+  @Retries.RetryTranslated
+  private SelectInputStream executeSelect(
+      final S3AReadOpContext readContext,
+      final S3ObjectAttributes objectAttributes,
+      final Configuration builderOptions,
+      final SelectObjectContentRequest request) throws IOException {
+
+    Path path = readContext.getPath();
+    if (readContext.getDstFileStatus().isDirectory()) {
+      throw new PathIOException(path.toString(),
+          "Can't select " + path
+          + " because it is a directory");
+    }
+    boolean sqlInErrors = builderOptions.getBoolean(SELECT_ERRORS_INCLUDE_SQL,
+        errorsIncludeSql);
+    String expression = request.getExpression();
+    final String errorText = sqlInErrors ? expression : "Select";
+    if (sqlInErrors) {
+      LOG.info("Issuing SQL request {}", expression);
+    }
+    return new SelectInputStream(readContext,
+        objectAttributes,
+        operations.select(path, request, errorText));
+  }
+
+  /**
+   * Build the select request from the configuration built up
+   * in {@code S3AFileSystem.openFile(Path)} and the default
+   * options in the cluster configuration.
+   *
+   * Options are picked up in the following order.
+   * <ol>
+   *   <li> Options in {@code openFileOptions}.</li>
+   *   <li> Options in the owning filesystem configuration.</li>
+   *   <li>The default values in {@link SelectConstants}</li>
+   * </ol>
+   *
+   * @param request request to build up
+   * @param expression SQL expression
+   * @param builderOptions the options which came in from the openFile builder.
+   * @throws IllegalArgumentException if an option is somehow invalid.
+   * @throws IOException if an option is somehow invalid.
+   */
+  void buildRequest(
+      final SelectObjectContentRequest request,
+      final String expression,
+      final Configuration builderOptions)
+      throws IllegalArgumentException, IOException {
+    Preconditions.checkArgument(StringUtils.isNotEmpty(expression),
+        "No expression provided in parameter " + SELECT_SQL);
+
+    final Configuration ownerConf = operations.getConf();
+
+
+    String inputFormat = builderOptions.get(SELECT_INPUT_FORMAT,
+        SELECT_FORMAT_CSV).toLowerCase(Locale.ENGLISH);
+    Preconditions.checkArgument(SELECT_FORMAT_CSV.equals(inputFormat),
+        "Unsupported input format %s", inputFormat);
+    String outputFormat = builderOptions.get(SELECT_OUTPUT_FORMAT,
+        SELECT_FORMAT_CSV)
+        .toLowerCase(Locale.ENGLISH);
+    Preconditions.checkArgument(SELECT_FORMAT_CSV.equals(outputFormat),
+        "Unsupported output format %s", outputFormat);
+
+    request.setExpressionType(ExpressionType.SQL);
+    request.setExpression(expandBackslashChars(expression));
+
+    InputSerialization inputSerialization = buildCsvInputRequest(ownerConf,
+        builderOptions);
+    String compression = opt(builderOptions,
+        ownerConf,
+        SELECT_INPUT_COMPRESSION,
+        COMPRESSION_OPT_NONE,
+        true).toUpperCase(Locale.ENGLISH);
+    if (isNotEmpty(compression)) {
+      inputSerialization.setCompressionType(compression);
+    }
+    request.setInputSerialization(inputSerialization);
+
+    request.setOutputSerialization(buildCSVOutput(ownerConf, builderOptions));
+
+  }
+
+  /**
+   * Build the CSV input request.
+   * @param ownerConf FS owner configuration
+   * @param builderOptions options on the specific request
+   * @return the constructed request
+   * @throws IllegalArgumentException argument failure
+   * @throws IOException validation failure
+   */
+  public InputSerialization buildCsvInputRequest(
+      final Configuration ownerConf,
+      final Configuration builderOptions)
+      throws IllegalArgumentException, IOException {
+
+    String headerInfo = opt(builderOptions,
+        ownerConf,
+        CSV_INPUT_HEADER,
+        CSV_INPUT_HEADER_OPT_DEFAULT,
+        true).toUpperCase(Locale.ENGLISH);
+    String commentMarker = xopt(builderOptions,
+        ownerConf,
+        CSV_INPUT_COMMENT_MARKER,
+        CSV_INPUT_COMMENT_MARKER_DEFAULT);
+    String fieldDelimiter = xopt(builderOptions,
+        ownerConf,
+        CSV_INPUT_INPUT_FIELD_DELIMITER,
+        CSV_INPUT_FIELD_DELIMITER_DEFAULT);
+    String recordDelimiter = xopt(builderOptions,
+        ownerConf,
+        CSV_INPUT_RECORD_DELIMITER,
+        CSV_INPUT_RECORD_DELIMITER_DEFAULT);
+    String quoteCharacter = xopt(builderOptions,
+        ownerConf,
+        CSV_INPUT_QUOTE_CHARACTER,
+        CSV_INPUT_QUOTE_CHARACTER_DEFAULT);
+    String quoteEscapeCharacter = xopt(builderOptions,
+        ownerConf,
+        CSV_INPUT_QUOTE_ESCAPE_CHARACTER,
+        CSV_INPUT_QUOTE_ESCAPE_CHARACTER_DEFAULT);
+
+    // CSV input
+    CSVInput csv = new CSVInput();
+    csv.setFieldDelimiter(fieldDelimiter);
+    csv.setRecordDelimiter(recordDelimiter);
+    csv.setComments(commentMarker);
+    csv.setQuoteCharacter(quoteCharacter);
+    if (StringUtils.isNotEmpty(quoteEscapeCharacter)) {
+      csv.setQuoteEscapeCharacter(quoteEscapeCharacter);
+    }
+    csv.setFileHeaderInfo(headerInfo);
+
+    InputSerialization inputSerialization = new InputSerialization();
+    inputSerialization.setCsv(csv);
+
+    return inputSerialization;
+
+  }
+
+  /**
+   * Build CSV output for a request.
+   * @param ownerConf FS owner configuration
+   * @param builderOptions options on the specific request
+   * @return the constructed request
+   * @throws IllegalArgumentException argument failure
+   * @throws IOException validation failure
+   */
+  public OutputSerialization buildCSVOutput(
+      final Configuration ownerConf,
+      final Configuration builderOptions)
+      throws IllegalArgumentException, IOException {
+    String fieldDelimiter = xopt(builderOptions,
+        ownerConf,
+        CSV_OUTPUT_FIELD_DELIMITER,
+        CSV_OUTPUT_FIELD_DELIMITER_DEFAULT);
+    String recordDelimiter = xopt(builderOptions,
+        ownerConf,
+        CSV_OUTPUT_RECORD_DELIMITER,
+        CSV_OUTPUT_RECORD_DELIMITER_DEFAULT);
+    String quoteCharacter = xopt(builderOptions,
+        ownerConf,
+        CSV_OUTPUT_QUOTE_CHARACTER,
+        CSV_OUTPUT_QUOTE_CHARACTER_DEFAULT);
+    String quoteEscapeCharacter = xopt(builderOptions,
+        ownerConf,
+        CSV_OUTPUT_QUOTE_ESCAPE_CHARACTER,
+        CSV_OUTPUT_QUOTE_ESCAPE_CHARACTER_DEFAULT);
+    String quoteFields = xopt(builderOptions,
+        ownerConf,
+        CSV_OUTPUT_QUOTE_FIELDS,
+        CSV_OUTPUT_QUOTE_FIELDS_ALWAYS).toUpperCase(Locale.ENGLISH);
+
+    // output is CSV, always
+    OutputSerialization outputSerialization
+        = new OutputSerialization();
+    CSVOutput csvOut = new CSVOutput();
+    csvOut.setQuoteCharacter(quoteCharacter);
+    csvOut.setQuoteFields(
+        QuoteFields.fromValue(quoteFields));
+    csvOut.setFieldDelimiter(fieldDelimiter);
+    csvOut.setRecordDelimiter(recordDelimiter);
+    if (!quoteEscapeCharacter.isEmpty()) {
+      csvOut.setQuoteEscapeCharacter(quoteEscapeCharacter);
+    }
+
+    outputSerialization.setCsv(csvOut);
+    return outputSerialization;
+  }
+
+  /**
+   * Stringify the given SelectObjectContentRequest, as its
+   * toString() operator doesn't.
+   * @param request request to convert to a string
+   * @return a string to print. Does not contain secrets.
+   */
+  public static String toString(final SelectObjectContentRequest request) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("SelectObjectContentRequest{")
+        .append("bucket name=").append(request.getBucketName())
+        .append("; key=").append(request.getKey())
+        .append("; expressionType=").append(request.getExpressionType())
+        .append("; expression=").append(request.getExpression());
+    InputSerialization input = request.getInputSerialization();
+    if (input != null) {
+      sb.append("; Input")
+          .append(input.toString());
+    } else {
+      sb.append("; Input Serialization: none");
+    }
+    OutputSerialization out = request.getOutputSerialization();
+    if (out != null) {
+      sb.append("; Output")
+          .append(out.toString());
+    } else {
+      sb.append("; Output Serialization: none");
+    }
+    return sb.append("}").toString();
+  }
+
+  /**
+   * Resolve an option.
+   * @param builderOptions the options which came in from the openFile builder.
+   * @param fsConf configuration of the owning FS.
+   * @param base base option (no s3a: prefix)
+   * @param defVal default value. Must not be null.
+   * @param trim should the result be trimmed.
+   * @return the possibly trimmed value.
+   */
+  static String opt(Configuration builderOptions,
+      Configuration fsConf,
+      String base,
+      String defVal,
+      boolean trim) {
+    String r = builderOptions.get(base, fsConf.get(base, defVal));
+    return trim ? r.trim() : r;
+  }
+
+  /**
+   * Get an option with backslash arguments transformed.
+   * These are not trimmed, so whitespace is significant.
+   * @param selectOpts options in the select call
+   * @param fsConf filesystem conf
+   * @param base base option name
+   * @param defVal default value
+   * @return the transformed value
+   */
+  static String xopt(Configuration selectOpts,
+      Configuration fsConf,
+      String base,
+      String defVal) {
+    return expandBackslashChars(
+        opt(selectOpts, fsConf, base, defVal, false));
+  }
+
+  /**
+   * Perform escaping.
+   * @param src source string.
+   * @return the replaced value
+   */
+  static String expandBackslashChars(String src) {
+    return src.replace("\\n", "\n")
+        .replace("\\\"", "\"")
+        .replace("\\t", "\t")
+        .replace("\\r", "\r")
+        .replace("\\\"", "\"")
+        // backslash substitution must come last
+        .replace("\\\\", "\\");
+  }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java
new file mode 100644
index 0000000..d74411d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java
@@ -0,0 +1,296 @@
+/*
+ * 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.select;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Options related to S3 Select.
+ *
+ * These options are set for the entire filesystem unless overridden
+ * as an option in the URI
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class SelectConstants {
+
+  public static final String SELECT_UNSUPPORTED = "S3 Select is not supported";
+
+  private SelectConstants() {
+  }
+
+  public static final String FS_S3A_SELECT = "fs.s3a.select.";
+
+
+  /**
+   * This is the big SQL expression: {@value}.
+   * When used in an open() call, switch to a select operation.
+   * This is only used in the open call, never in a filesystem configuration.
+   */
+  public static final String SELECT_SQL = FS_S3A_SELECT + "sql";
+
+  /**
+   * Does the FS Support S3 Select?
+   * Value: {@value}.
+   */
+  public static final String S3_SELECT_CAPABILITY = "s3a:fs.s3a.select.sql";
+
+  /**
+   * Flag: is S3 select enabled?
+   * Value: {@value}.
+   */
+  public static final String FS_S3A_SELECT_ENABLED = FS_S3A_SELECT
+      + "enabled";
+
+  /**
+   * Input format for data.
+   * Value: {@value}.
+   */
+  public static final String SELECT_INPUT_FORMAT =
+      "fs.s3a.select.input.format";
+
+  /**
+   * Output format for data -that is, what the results are generated
+   * as.
+   * Value: {@value}.
+   */
+  public static final String SELECT_OUTPUT_FORMAT =
+      "fs.s3a.select.output.format";
+
+  /**
+   * CSV as an input or output format: {@value}.
+   */
+  public static final String SELECT_FORMAT_CSV = "csv";
+
+  /**
+   * JSON as an input or output format: {@value}.
+   */
+  public static final String SELECT_FORMAT_JSON = "json";
+
+  /**
+   * Should Select errors include the SQL statement?
+   * It is easier to debug but a security risk if the exceptions
+   * ever get printed/logged and the query contains secrets.
+   */
+  public static final String SELECT_ERRORS_INCLUDE_SQL =
+      FS_S3A_SELECT + "errors.include.sql";
+
+  /**
+   * How is the input compressed? This applies to all formats.
+   * Value: {@value}.
+   */
+  public static final String SELECT_INPUT_COMPRESSION = FS_S3A_SELECT
+      + "input.compression";
+
+  /**
+   * No compression.
+   * Value: {@value}.
+   */
+  public static final String COMPRESSION_OPT_NONE = "none";
+
+  /**
+   * Gzipped.
+   * Value: {@value}.
+   */
+  public static final String COMPRESSION_OPT_GZIP = "gzip";
+
+  /**
+   * Prefix for all CSV input options.
+   * Value: {@value}.
+   */
+  public static final String FS_S3A_SELECT_INPUT_CSV =
+      "fs.s3a.select.input.csv.";
+
+  /**
+   * Prefix for all CSV output options.
+   * Value: {@value}.
+   */
+  public static final String FS_S3A_SELECT_OUTPUT_CSV =
+      "fs.s3a.select.output.csv.";
+
+  /**
+   * String which indicates the row is actually a comment.
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_COMMENT_MARKER =
+      FS_S3A_SELECT_INPUT_CSV + "comment.marker";
+
+  /**
+   * Default marker.
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_COMMENT_MARKER_DEFAULT = "#";
+
+  /**
+   * Record delimiter. CR, LF, etc.
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_RECORD_DELIMITER =
+      FS_S3A_SELECT_INPUT_CSV + "record.delimiter";
+
+  /**
+   * Default delimiter
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_RECORD_DELIMITER_DEFAULT = "\n";
+
+  /**
+   * Field delimiter.
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_INPUT_FIELD_DELIMITER =
+      FS_S3A_SELECT_INPUT_CSV + "field.delimiter";
+
+  /**
+   * Default field delimiter.
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_FIELD_DELIMITER_DEFAULT = ",";
+
+  /**
+   * Quote Character.
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_QUOTE_CHARACTER =
+      FS_S3A_SELECT_INPUT_CSV + "quote.character";
+
+  /**
+   * Default Quote Character.
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_QUOTE_CHARACTER_DEFAULT = "\"";
+
+  /**
+   * Character to escape quotes.
+   * If empty: no escaping.
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_QUOTE_ESCAPE_CHARACTER =
+      FS_S3A_SELECT_INPUT_CSV + "quote.escape.character";
+
+  /**
+   * Default quote escape character.
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_QUOTE_ESCAPE_CHARACTER_DEFAULT = "\\";
+
+  /**
+   * How should headers be used?
+   * Value: {@value}.
+   */
+  public static final String CSV_INPUT_HEADER =
+      FS_S3A_SELECT_INPUT_CSV + "header";
+
+  /**
+   * No header: first row is data.
+   * Value: {@value}.
+   */
+  public static final String CSV_HEADER_OPT_NONE = "none";
+
+  /**
+   * Ignore the header.
+   * Value: {@value}.
+   */
+  public static final String CSV_HEADER_OPT_IGNORE = "ignore";
+
+  /**
+   * Use the header.
+   * Value: {@value}.
+   */
+  public static final String CSV_HEADER_OPT_USE = "use";
+
+  /**
+   * Default header mode: {@value}.
+   */
+  public static final String CSV_INPUT_HEADER_OPT_DEFAULT =
+      CSV_HEADER_OPT_IGNORE;
+
+  /**
+   * Record delimiter. CR, LF, etc.
+   * Value: {@value}.
+   */
+  public static final String CSV_OUTPUT_RECORD_DELIMITER =
+      FS_S3A_SELECT_OUTPUT_CSV + "record.delimiter";
+
+  /**
+   * Default delimiter
+   * Value: {@value}.
+   */
+  public static final String CSV_OUTPUT_RECORD_DELIMITER_DEFAULT = "\n";
+
+  /**
+   * Field delimiter.
+   * Value: {@value}.
+   */
+  public static final String CSV_OUTPUT_FIELD_DELIMITER =
+      FS_S3A_SELECT_OUTPUT_CSV + "field.delimiter";
+
+  /**
+   * Default field delimiter.
+   * Value: {@value}.
+   */
+  public static final String CSV_OUTPUT_FIELD_DELIMITER_DEFAULT = ",";
+
+  /**
+   * Quote Character.
+   * Value: {@value}.
+   */
+  public static final String CSV_OUTPUT_QUOTE_CHARACTER =
+      FS_S3A_SELECT_OUTPUT_CSV + "quote.character";
+
+  /**
+   * Default Quote Character.
+   * Value: {@value}.
+   */
+  public static final String CSV_OUTPUT_QUOTE_CHARACTER_DEFAULT = "\"";
+
+  /**
+   * Should CSV fields be quoted?
+   * One of : ALWAYS, ASNEEDED
+   * Value: {@value}.
+   */
+  public static final String CSV_OUTPUT_QUOTE_FIELDS =
+      FS_S3A_SELECT_OUTPUT_CSV + "quote.fields";
+
+  /**
+   * Output quotation policy (default): {@value}.
+   */
+  public static final String CSV_OUTPUT_QUOTE_FIELDS_ALWAYS = "always";
+
+  /**
+   * Output quotation policy: {@value}.
+   */
+  public static final String CSV_OUTPUT_QUOTE_FIELDS_AS_NEEEDED = "asneeded";
+
+  /**
+   * Character to escape quotes.
+   * If empty: no escaping.
+   * Value: {@value}.
+   */
+  public static final String CSV_OUTPUT_QUOTE_ESCAPE_CHARACTER =
+      FS_S3A_SELECT_OUTPUT_CSV + "quote.escape.character";
+
+  /**
+   * Default quote escape character.
+   * Value: {@value}.
+   */
+  public static final String CSV_OUTPUT_QUOTE_ESCAPE_CHARACTER_DEFAULT = "";
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
new file mode 100644
index 0000000..f4bd8d1
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
@@ -0,0 +1,457 @@
+/*
+ * 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.select;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.amazonaws.AbortedException;
+import com.amazonaws.services.s3.model.SelectObjectContentEvent;
+import com.amazonaws.services.s3.model.SelectObjectContentEventVisitor;
+import com.amazonaws.services.s3.model.SelectObjectContentResult;
+import com.amazonaws.services.s3.model.SelectRecordsInputStream;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CanSetReadahead;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.S3AReadOpContext;
+import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
+import org.apache.hadoop.io.IOUtils;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.commons.lang3.StringUtils.isNotEmpty;
+import static org.apache.hadoop.fs.s3a.Invoker.once;
+import static org.apache.hadoop.fs.s3a.S3AInputStream.validateReadahead;
+
+/**
+ * An input stream for S3 Select return values.
+ * This is simply an end-to-end GET request, without any
+ * form of seek or recovery from connectivity failures.
+ *
+ * Currently only seek and positioned read operations on the current
+ * location are supported.
+ *
+ * The normal S3 input counters are updated by this stream.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SelectInputStream extends FSInputStream implements
+    CanSetReadahead {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SelectInputStream.class);
+
+  public static final String SEEK_UNSUPPORTED = "seek()";
+
+  /**
+   * Same set of arguments as for an S3AInputStream.
+   */
+  private final S3ObjectAttributes objectAttributes;
+
+  /**
+   * Tracks the current position.
+   */
+  private AtomicLong pos = new AtomicLong(0);
+
+  /**
+   * Closed flag.
+   */
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+
+  /**
+   * Did the read complete successfully?
+   */
+  private final AtomicBoolean completedSuccessfully = new AtomicBoolean(false);
+
+  /**
+   * Abortable response stream.
+   * This is guaranteed to never be null.
+   */
+  private final SelectRecordsInputStream wrappedStream;
+
+  private final String bucket;
+
+  private final String key;
+
+  private final String uri;
+
+  private final S3AReadOpContext readContext;
+
+  private final S3AInstrumentation.InputStreamStatistics streamStatistics;
+
+  private long readahead;
+
+  /**
+   * Create the stream.
+   * The read attempt is initiated immediately.
+   * @param readContext read context
+   * @param objectAttributes object attributes from a HEAD request
+   * @param selectResponse response from the already executed call
+   * @throws IOException failure
+   */
+  @Retries.OnceTranslated
+  public SelectInputStream(
+      final S3AReadOpContext readContext,
+      final S3ObjectAttributes objectAttributes,
+      final SelectObjectContentResult selectResponse) throws IOException {
+    Preconditions.checkArgument(isNotEmpty(objectAttributes.getBucket()),
+        "No Bucket");
+    Preconditions.checkArgument(isNotEmpty(objectAttributes.getKey()),
+        "No Key");
+    this.objectAttributes = objectAttributes;
+    this.bucket = objectAttributes.getBucket();
+    this.key = objectAttributes.getKey();
+    this.uri = "s3a://" + this.bucket + "/" + this.key;
+    this.readContext = readContext;
+    this.readahead = readContext.getReadahead();
+    this.streamStatistics = readContext.getInstrumentation()
+        .newInputStreamStatistics();
+    SelectRecordsInputStream stream = once(
+        "S3 Select",
+        uri,
+        () -> selectResponse.getPayload()
+            .getRecordsInputStream(new SelectObjectContentEventVisitor() {
+              @Override
+              public void visit(final SelectObjectContentEvent.EndEvent event) {
+                LOG.debug("Completed successful S3 select read from {}", uri);
+                completedSuccessfully.set(true);
+              }
+            }));
+    this.wrappedStream = checkNotNull(stream);
+    // this stream is already opened, so mark as such in the statistics.
+    streamStatistics.streamOpened();
+  }
+
+  @Override
+  public void close() throws IOException {
+    long skipped = 0;
+    boolean aborted = false;
+    if (!closed.getAndSet(true)) {
+      try {
+        // set up for aborts.
+        // if we know the available amount > readahead. Abort.
+        //
+        boolean shouldAbort = wrappedStream.available() > readahead;
+        if (!shouldAbort) {
+          // read our readahead range worth of data
+          skipped = wrappedStream.skip(readahead);
+          shouldAbort = wrappedStream.read() >= 0;
+        }
+        // now, either there is data left or not.
+        if (shouldAbort) {
+          // yes, more data. Abort and add this fact to the stream stats
+          aborted = true;
+          wrappedStream.abort();
+        }
+      } catch (IOException | AbortedException e) {
+        LOG.debug("While closing stream", e);
+      } finally {
+        IOUtils.cleanupWithLogger(LOG, wrappedStream);
+        streamStatistics.streamClose(aborted, skipped);
+        streamStatistics.close();
+        super.close();
+      }
+    }
+  }
+
+  /**
+   * Verify that the input stream is open. Non blocking; this gives
+   * the last state of the atomic {@link #closed} field.
+   * @throws PathIOException if the connection is closed.
+   */
+  private void checkNotClosed() throws IOException {
+    if (closed.get()) {
+      throw new PathIOException(uri, FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    checkNotClosed();
+    return wrappedStream.available();
+  }
+
+  @Override
+  @Retries.OnceTranslated
+  public synchronized long skip(final long n) throws IOException {
+    checkNotClosed();
+    long skipped = once("skip", uri, () -> wrappedStream.skip(n));
+    pos.addAndGet(skipped);
+    // treat as a forward skip for stats
+    streamStatistics.seekForwards(skipped);
+    return skipped;
+  }
+
+  @Override
+  public long getPos() {
+    return pos.get();
+  }
+
+  /**
+   * Set the readahead.
+   * @param readahead The readahead to use.  null means to use the default.
+   */
+  @Override
+  public void setReadahead(Long readahead) {
+    this.readahead = validateReadahead(readahead);
+  }
+
+  /**
+   * Get the current readahead value.
+   * @return the readahead
+   */
+  public long getReadahead() {
+    return readahead;
+  }
+
+  /**
+   * Read a byte. There's no attempt to recover, but AWS-SDK exceptions
+   * such as {@code SelectObjectContentEventException} are translated into
+   * IOExceptions.
+   * @return a byte read or -1 for an end of file.
+   * @throws IOException failure.
+   */
+  @Override
+  @Retries.OnceTranslated
+  public synchronized int read() throws IOException {
+    checkNotClosed();
+    int byteRead;
+    try {
+      byteRead = once("read()", uri, () -> wrappedStream.read());
+    } catch (EOFException e) {
+      // this could be one of: end of file, some IO failure
+      if (completedSuccessfully.get()) {
+        // read was successful
+        return -1;
+      } else {
+        // the stream closed prematurely
+        LOG.info("Reading of S3 Select data from {} failed before all results "
+            + " were generated.", uri);
+        streamStatistics.readException();
+        throw new PathIOException(uri,
+            "Read of S3 Select data did not complete");
+      }
+    }
+
+    if (byteRead >= 0) {
+      incrementBytesRead(1);
+    }
+    return byteRead;
+  }
+
+  @SuppressWarnings("NullableProblems")
+  @Override
+  @Retries.OnceTranslated
+  public synchronized int read(final byte[] buf, final int off, final int len)
+      throws IOException {
+    checkNotClosed();
+    validatePositionedReadArgs(pos.get(), buf, off, len);
+    if (len == 0) {
+      return 0;
+    }
+
+    int bytesRead;
+    try {
+      streamStatistics.readOperationStarted(pos.get(), len);
+      bytesRead = wrappedStream.read(buf, off, len);
+    } catch (EOFException e) {
+      streamStatistics.readException();
+      // the base implementation swallows EOFs.
+      return -1;
+    }
+
+    incrementBytesRead(bytesRead);
+    streamStatistics.readOperationCompleted(len, bytesRead);
+    return bytesRead;
+  }
+
+  /**
+   * Forward seeks are supported, but not backwards ones.
+   * Forward seeks are implemented using read, so
+   * means that long-distance seeks will be (literally) expensive.
+   *
+   * @param newPos new seek position.
+   * @throws PathIOException Backwards seek attempted.
+   * @throws EOFException attempt to seek past the end of the stream.
+   * @throws IOException IO failure while skipping bytes
+   */
+  @Override
+  @Retries.OnceTranslated
+  public synchronized void seek(long newPos) throws IOException {
+    long current = getPos();
+    long distance = newPos - current;
+    if (distance < 0) {
+      throw unsupported(SEEK_UNSUPPORTED
+          + " backwards from " + current + " to " + newPos);
+    }
+    if (distance == 0) {
+      LOG.debug("ignoring seek to current position.");
+    } else {
+      // the complicated one: Forward seeking. Useful for split files.
+      LOG.debug("Forward seek by reading {} bytes", distance);
+      long bytesSkipped = 0;
+      // read byte-by-byte, hoping that buffering will compensate for this.
+      // doing it this way ensures that the seek stops at exactly the right
+      // place. skip(len) can return a smaller value, at which point
+      // it's not clear what to do.
+      while(distance > 0) {
+        int r = read();
+        if (r == -1) {
+          // reached an EOF too early
+          throw new EOFException("Seek to " + newPos
+              + " reached End of File at offset " + getPos());
+        }
+        distance--;
+        bytesSkipped++;
+      }
+      // read has finished.
+      streamStatistics.seekForwards(bytesSkipped);
+    }
+  }
+
+  /**
+   * Build an exception to raise when an operation is not supported here.
+   * @param action action which is unsupported.
+   * @return an exception to throw.
+   */
+  protected PathIOException unsupported(final String action) {
+    return new PathIOException(
+        String.format("s3a://%s/%s", bucket, key),
+        action + " not supported");
+  }
+
+  @Override
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    return false;
+  }
+
+  // Not supported.
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+
+  @SuppressWarnings("NonSynchronizedMethodOverridesSynchronizedMethod")
+  @Override
+  public void mark(int readLimit) {
+    // Do nothing
+  }
+
+  @SuppressWarnings("NonSynchronizedMethodOverridesSynchronizedMethod")
+  @Override
+  public void reset() throws IOException {
+    throw unsupported("Mark");
+  }
+
+  /**
+   * Aborts the IO.
+   */
+  public void abort() {
+    if (!closed.get()) {
+      LOG.debug("Aborting");
+      wrappedStream.abort();
+    }
+  }
+
+  /**
+   * Read at a specific position.
+   * Reads at a position earlier than the current {@link #getPos()} position
+   * will fail with a {@link PathIOException}. See {@link #seek(long)}.
+   * Unlike the base implementation <i>And the requirements of the filesystem
+   * specification, this updates the stream position as returned in
+   * {@link #getPos()}.</i>
+   * @param position offset in the stream.
+   * @param buffer buffer to read in to.
+   * @param offset offset within the buffer
+   * @param length amount of data to read.
+   * @return the result.
+   * @throws PathIOException Backwards seek attempted.
+   * @throws EOFException attempt to seek past the end of the stream.
+   * @throws IOException IO failure while seeking in the stream or reading data.
+   */
+  @Override
+  public int read(final long position,
+      final byte[] buffer,
+      final int offset,
+      final int length)
+      throws IOException {
+    // maybe seek forwards to the position.
+    seek(position);
+    return read(buffer, offset, length);
+  }
+
+  /**
+   * Increment the bytes read counter if there is a stats instance
+   * and the number of bytes read is more than zero.
+   * This also updates the {@link #pos} marker by the same value.
+   * @param bytesRead number of bytes read
+   */
+  private void incrementBytesRead(long bytesRead) {
+    if (bytesRead > 0) {
+      pos.addAndGet(bytesRead);
+    }
+    streamStatistics.bytesRead(bytesRead);
+    if (readContext.getStats() != null && bytesRead > 0) {
+      readContext.getStats().incrementBytesRead(bytesRead);
+    }
+  }
+
+  /**
+   * Get the Stream statistics.
+   * @return the statistics for this stream.
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public S3AInstrumentation.InputStreamStatistics getS3AStreamStatistics() {
+    return streamStatistics;
+  }
+
+  /**
+   * String value includes statistics as well as stream state.
+   * <b>Important: there are no guarantees as to the stability
+   * of this value.</b>
+   * @return a string value for printing in logs/diagnostics
+   */
+  @Override
+  @InterfaceStability.Unstable
+  public String toString() {
+    String s = streamStatistics.toString();
+    synchronized (this) {
+      final StringBuilder sb = new StringBuilder(
+          "SelectInputStream{");
+      sb.append(uri);
+      sb.append("; state ").append(!closed.get() ? "open" : "closed");
+      sb.append("; pos=").append(getPos());
+      sb.append("; readahead=").append(readahead);
+      sb.append('\n').append(s);
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+}
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
new file mode 100644
index 0000000..8c87694
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectTool.java
@@ -0,0 +1,355 @@
+/*
+ * 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.select;
+
+import java.io.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Locale;
+import java.util.Optional;
+import java.util.Scanner;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.IOUtils;
+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.fs.impl.FutureIOSupport;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.commit.Duration;
+import org.apache.hadoop.fs.s3a.commit.DurationInfo;
+import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool;
+import org.apache.hadoop.fs.shell.CommandFormat;
+import org.apache.hadoop.util.ExitUtil;
+
+import static org.apache.commons.lang3.StringUtils.isNotEmpty;
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
+import static org.apache.hadoop.service.launcher.LauncherExitCodes.*;
+import static org.apache.hadoop.fs.s3a.select.SelectConstants.*;
+
+/**
+ * This is a CLI tool for the select operation, which is available
+ * through the S3Guard command.
+ *
+ * Usage:
+ * <pre>
+ *   hadoop s3guard select [options] Path Statement
+ * </pre>
+ */
+public class SelectTool extends S3GuardTool {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SelectTool.class);
+
+  public static final String NAME = "select";
+
+  public static final String PURPOSE = "make an S3 Select call";
+
+  private static final String USAGE = NAME
+      + " [OPTIONS]"
+      + " [-limit rows]"
+      + " [-header (use|none|ignore)]"
+      + " [-out path]"
+      + " [-expected rows]"
+      + " [-compression (gzip|bzip2|none)]"
+      + " [-inputformat csv]"
+      + " [-outputformat csv]"
+      + " <PATH> <SELECT QUERY>\n"
+      + "\t" + PURPOSE + "\n\n";
+
+  public static final String OPT_COMPRESSION = "compression";
+
+  public static final String OPT_EXPECTED = "expected";
+
+  public static final String OPT_HEADER = "header";
+
+  public static final String OPT_INPUTFORMAT = "inputformat";
+
+  public static final String OPT_LIMIT = "limit";
+
+  public static final String OPT_OUTPUT = "out";
+
+  public static final String OPT_OUTPUTFORMAT = "inputformat";
+
+  static final String TOO_FEW_ARGUMENTS = "Too few arguments";
+
+  static final String WRONG_FILESYSTEM = "Wrong filesystem for ";
+
+  static final String SELECT_IS_DISABLED = "S3 Select is disabled";
+
+  private Duration selectDuration;
+
+  private long bytesRead;
+
+  private long linesRead;
+
+  public SelectTool(Configuration conf) {
+    super(conf);
+    // read capacity.
+    getCommandFormat().addOptionWithValue(OPT_COMPRESSION);
+    getCommandFormat().addOptionWithValue(OPT_EXPECTED);
+    getCommandFormat().addOptionWithValue(OPT_HEADER);
+    getCommandFormat().addOptionWithValue(OPT_INPUTFORMAT);
+    getCommandFormat().addOptionWithValue(OPT_LIMIT);
+    getCommandFormat().addOptionWithValue(OPT_OUTPUT);
+    getCommandFormat().addOptionWithValue(OPT_OUTPUTFORMAT);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  @Override
+  public String getUsage() {
+    return USAGE;
+  }
+
+  public Duration getSelectDuration() {
+    return selectDuration;
+  }
+
+  public long getBytesRead() {
+    return bytesRead;
+  }
+
+  /**
+   * Number of lines read, when printing to the console.
+   * @return line count. 0 if writing direct to a file.
+   */
+  public long getLinesRead() {
+    return linesRead;
+  }
+
+  private int parseNaturalInt(String option, String value) {
+    try {
+      int r = Integer.parseInt(value);
+      if (r < 0) {
+        throw invalidArgs("Negative value for option %s : %s", option, value);
+      }
+      return r;
+    } catch (NumberFormatException e) {
+      throw invalidArgs("Invalid number for option %s : %s", option, value);
+    }
+  }
+
+  private Optional<String> getOptValue(String key) {
+    String value = getCommandFormat().getOptValue(key);
+    return isNotEmpty(value) ? Optional.of(value): Optional.empty();
+  }
+
+  private Optional<Integer> getIntValue(String key) {
+    Optional<String> v = getOptValue(key);
+    return v.map(i -> parseNaturalInt(key, i));
+  }
+
+  /**
+   * Execute the select operation.
+   * @param args argument list
+   * @param out output stream
+   * @return an exit code
+   * @throws IOException IO failure
+   * @throws ExitUtil.ExitException managed failure
+   */
+  public int run(String[] args, PrintStream out)
+      throws IOException, ExitUtil.ExitException {
+    final List<String> parsedArgs;
+    try {
+      parsedArgs = parseArgs(args);
+    } catch (CommandFormat.UnknownOptionException e) {
+      errorln(getUsage());
+      throw new ExitUtil.ExitException(EXIT_USAGE, e.getMessage(), e);
+    }
+    if (parsedArgs.size() < 2) {
+      errorln(getUsage());
+      throw new ExitUtil.ExitException(EXIT_USAGE, TOO_FEW_ARGUMENTS);
+    }
+
+    // read mandatory arguments
+    final String file = parsedArgs.get(0);
+    final Path path = new Path(file);
+
+    String expression = parsedArgs.get(1);
+
+    println(out, "selecting file %s with query %s",
+        path, expression);
+
+    // and the optional arguments to adjust the configuration.
+    final Optional<String> header = getOptValue(OPT_HEADER);
+    header.ifPresent(h -> println(out, "Using header option %s", h));
+
+    Path destPath = getOptValue(OPT_OUTPUT).map(
+        output -> {
+          println(out, "Saving output to %s", output);
+          return new Path(output);
+        }).orElse(null);
+    final boolean toConsole = destPath == null;
+
+    // expected lines are only checked if empty
+    final Optional<Integer> expectedLines = toConsole
+        ? getIntValue(OPT_EXPECTED)
+        : Optional.empty();
+
+    final Optional<Integer> limit = getIntValue(OPT_LIMIT);
+    if (limit.isPresent()) {
+      final int l = limit.get();
+      println(out, "Using line limit %s", l);
+      if (expression.toLowerCase(Locale.ENGLISH).contains(" limit ")) {
+        println(out, "line limit already specified in SELECT expression");
+      } else {
+        expression = expression + " LIMIT " + l;
+      }
+    }
+
+    // now bind to the filesystem.
+    FileSystem fs = path.getFileSystem(getConf());
+    if (!(fs instanceof S3AFileSystem)) {
+      throw new ExitUtil.ExitException(EXIT_SERVICE_UNAVAILABLE,
+          WRONG_FILESYSTEM + file + ": got " + fs);
+    }
+    setFilesystem((S3AFileSystem) fs);
+
+    if (!getFilesystem().hasCapability(S3_SELECT_CAPABILITY)) {
+      // capability disabled
+      throw new ExitUtil.ExitException(EXIT_SERVICE_UNAVAILABLE,
+          SELECT_IS_DISABLED + " for " + file);
+    }
+    linesRead = 0;
+
+    selectDuration = new Duration();
+
+    // open and scan the stream.
+    final FutureDataInputStreamBuilder builder = fs.openFile(path)
+        .must(SELECT_SQL, expression);
+
+    header.ifPresent(h -> builder.must(CSV_INPUT_HEADER, h));
+
+    getOptValue(OPT_COMPRESSION).ifPresent(compression ->
+        builder.must(SELECT_INPUT_COMPRESSION,
+          compression.toUpperCase(Locale.ENGLISH)));
+
+    getOptValue(OPT_INPUTFORMAT).ifPresent(opt -> {
+      if (!"csv".equalsIgnoreCase(opt)) {
+        throw invalidArgs("Unsupported input format %s", opt);
+      }
+    });
+    getOptValue(OPT_OUTPUTFORMAT).ifPresent(opt -> {
+      if (!"csv".equalsIgnoreCase(opt)) {
+        throw invalidArgs("Unsupported output format %s", opt);
+      }
+    });
+    // turn on SQL error reporting.
+    builder.opt(SELECT_ERRORS_INCLUDE_SQL, true);
+
+    FSDataInputStream stream;
+    try(DurationInfo ignored =
+            new DurationInfo(LOG, "Selecting stream")) {
+      stream = FutureIOSupport.awaitFuture(builder.build());
+    } catch (FileNotFoundException e) {
+      // the source file is missing.
+      throw storeNotFound(e);
+    }
+    try {
+      if (toConsole) {
+        // logging to console
+        bytesRead = 0;
+        @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
+        Scanner scanner =
+            new Scanner(
+                new BufferedReader(
+                    new InputStreamReader(stream, StandardCharsets.UTF_8)));
+        scanner.useDelimiter("\n");
+        while (scanner.hasNextLine()) {
+          linesRead++;
+          String l = scanner.nextLine();
+          bytesRead += l.length() + 1;
+          println(out, "%s", l);
+        }
+      } else {
+        // straight dump of whole file; no line counting
+        FileSystem destFS = destPath.getFileSystem(getConf());
+        try(DurationInfo ignored =
+                new DurationInfo(LOG, "Copying File");
+            OutputStream destStream = destFS.createFile(destPath)
+                .overwrite(true)
+                .build()) {
+          bytesRead = IOUtils.copy(stream, destStream);
+        }
+      }
+
+      // close the stream.
+      // this will take time if there's a lot of data remaining
+      try (DurationInfo ignored =
+               new DurationInfo(LOG, "Closing stream")) {
+        stream.close();
+      }
+
+      // generate a meaningful result depending on the operation
+      String result = toConsole
+          ? String.format("%s lines", linesRead)
+          : String.format("%s bytes", bytesRead);
+
+      // print some statistics
+      selectDuration.finished();
+      println(out, "Read %s in time %s",
+          result, selectDuration.getDurationString());
+
+      println(out, "Bytes Read: %,d bytes", bytesRead);
+
+      println(out, "Bandwidth: %,.1f MiB/s",
+          bandwidthMBs(bytesRead, selectDuration.value()));
+
+    } finally {
+      cleanupWithLogger(LOG, stream);
+    }
+
+    LOG.debug("Statistics {}", stream);
+
+    expectedLines.ifPresent(l -> {
+      if (l != linesRead) {
+        throw exitException(EXIT_FAIL,
+            "Expected %d rows but the operation returned %d",
+            l, linesRead);
+      }
+    });
+    out.flush();
+    return EXIT_SUCCESS;
+  }
+
+  /**
+   * Work out the bandwidth in MB/s.
+   * @param bytes bytes
+   * @param durationMillisNS duration in nanos
+   * @return the number of megabytes/second of the recorded operation
+   */
+  public static double bandwidthMBs(long bytes, long durationMillisNS) {
+    return durationMillisNS > 0
+        ? (bytes / 1048576.0 * 1000 / durationMillisNS)
+        : 0;
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/package-info.java
new file mode 100644
index 0000000..6cca6f4
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/**
+ * Support for S3 Select.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.s3a.select;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md
new file mode 100644
index 0000000..a684c3a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md
@@ -0,0 +1,1100 @@
+<!---
+  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.
+-->
+
+# S3 Select
+
+**Experimental Feature**
+
+<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
+
+S3 Select is a feature for Amazon S3 introduced in April 2018. It allows for
+SQL-like SELECT expressions to be applied to files in some structured
+formats, including CSV and JSON.
+
+By performing the SELECT operation in the S3 storage infrastructure, the
+bandwidth requirements between S3 and the hosts making the request can be reduced.
+Along with latency, this bandwidth is often the limiting factor in processing
+data from S3, especially with larger CSV and JSON datasets.
+
+Apache Hadoop's S3A Client has experimental support for this feature, with the
+following warnings:
+
+* The filtering is being done in S3 itself. If the source files cannot be parsed,
+that's not something which can be fixed in Hadoop or layers above.
+* It is not currently supported by third party S3 implementations, and unlikely
+to be supported in future (the bandwidth constraints are less, so the value
+less compelling).
+* Performance *appears* best when the selection restricts the number of fields,
+and projected columns: the less data returned, the faster the response.
+* High-level support in tools such as Apache Hive and Spark will also be
+evolving. Nobody has ever written CSV connectors with predicate pushdown before.
+* The standard `FileInputFormat` readers of text (`LineRecordReader` etc) fail when the
+amount of data returned is less than they expect. For this reason, S3 Select
+*MUST NOT BE USED IN PRODUCTION MAPREDUCE JOBS*.
+
+## Currently Implemented Features
+
+* Ability to issue select queries on the command line.
+* Proof of concept support in MapReduce queries.
+* CSV input with/without compression.
+* CSV output.
+
+## Currently Unsupported
+
+* Production-ready integration with the standard FileInputFormat and
+Record Readers.
+* Non-CSV output.
+* JSON source files.
+* Structured source file formats like Apache Parquet.
+It's better here to directly use the Apache Spark, Hive, Impala, Flink or
+similar, which all use the latest ASF-supported libraries.
+
+## Enabling/Disabling S3 Select
+
+S3 Select is enabled by default:
+
+```xml
+<property>
+  <name>fs.s3a.select.enabled</name>
+  <value>true</value>
+  <description>Is S3 Select enabled?</description>
+</property>
+```
+
+To disable it, set the option `fs.s3a.select.enabled` to `false`.
+
+To probe to see if a FileSystem instance implements it,
+`StreamCapabilities.hasCapability("s3a:fs.s3a.select.sql")` will be true
+on an instance of the S3AFileSystem class if
+the version of Hadoop supports S3 Select, *and* it is enabled for that
+specific instance.
+
+If this call returns false, then S3 Select calls will fail.
+
+Rather than cast a filesystem to the `S3AFileSystem` class, cast it to
+`org.apache.hadoop.fs.StreamCapabilities`; a class which was added in Hadoop 2.9.
+This should result in less brittle code -and there is no need to have the
+`hadoop-aws` JAR on the classpath at compile time.
+
+```java
+/**
+ * Probe for a filesystem instance supporting S3 Select.
+ * @param fs filesystem
+ * @return true if the filesystem supports S3 Select.
+ */
+public static boolean hasS3SelectCapability(final FileSystem fs) {
+  return (fs instanceof StreamCapabilities)
+    && ((StreamCapabilities)fs).hasCapability("s3a:fs.s3a.select.sql");
+}
+```
+
+## Making S3 Select calls through the Hadoop APIs
+
+Applications can issue select queries through the Hadoop FileSystem/FileContext
+ APIs via the asynchronous `openFile()` call added in Hadoop 3.3.
+
+Use the `FileSystem.openFile(path)` or `FileContext.openFile(path)` methods
+command to get a builder class for the open operations, then
+set the mandatory s3 select options though multiple `must()` parameters.
+
+```java
+FileSystem.FSDataInputStreamBuilder builder =
+    filesystem.openFile("s3a://bucket/path-to-file.csv")
+        .must("fs.s3a.select.sql",
+            "SELECT * FROM S3OBJECT s WHERE s.\"odd\" = `TRUE`")
+        .must("fs.s3a.select.input.format", "CSV")
+        .must("fs.s3a.select.input.compression", "NONE")
+        .must("fs.s3a.select.input.csv.header", "use")
+        .must("fs.s3a.select.output.format", "CSV")
+        .must("fs.s3a.select.output.csv.field.delimiter", "\t")
+        .must("fs.s3a.select.output.csv.quote.character", "\"")
+        .must("fs.s3a.select.output.csv.quote.fields", "asneeded") ;
+CompletableFuture<FSDataInputStream> future = builder.build();
+try (FSDataInputStream select = future.get()) {
+    // process the output
+    byte[] bytes = new byte[8192];
+    int actual = select.read(bytes);
+}
+```
+
+When the Builder's `build()` call is made, if the FileSystem/FileContext
+instance does not recognize any of the mandatory options it will fail.
+The S3A connector does recognize them, and, if S3 Select has not been
+disabled, will issue the Select query against the object store.
+
+If the S3A connector has S3 Select disabled, it will fail with
+an `UnsupportedOperationException`.
+
+The `build()` call returns a `CompletableFuture<FSDataInputStream>`.
+This future retrieves the result of the select call, which is executed
+asynchronously in the S3A FileSystem instance's executor pool.
+
+Errors in the SQL, missing file, permission failures and suchlike
+will surface when the future is evaluated, *not the build call*.
+
+In the returned stream, seeking and positioned reads do not work as usual,
+because there are no absolute positions in the file to seek to.
+
+1. backwards `seek()` calls will raise a `PathIOException`.
+1. Forwards `seek()` calls will succeed, but only by reading and discarding
+bytes. This will be slow.
+1. All positioned read operations when the offset of the read is not the current position
+will raise a `PathIOException`.
+1. Positioned read operations when the offset of the read *is* current position
+   will succeed, but the position of the stream (as returned by `getPos()`)
+   will be updated. This is not compliant with the filesystem specification.
+
+This is all done as a best-effort attempt to support existing code which
+often uses `seek()` to move forward in a split file after opening,
+or does a series of positioned read calls.
+
+
+### seek() behavior on `SelectInputStream`
+
+The returned stream, of type `org.apache.hadoop.fs.s3a.select.SelectInputStream`,
+only supports forward `seek()` operations.
+
+A zero-byte seek operation is always valid:
+
+```java
+stream.seek(stream.getPos());
+```
+
+A negative seek operation will always fail:
+
+```java
+stream.seek(stream.getPos() - offset);
+```
+
+A forward seek operation will work, provided the final position is less
+than the total length of the stream:
+
+```java
+stream.seek(stream.getPos() + offset);
+```
+
+If it is past the end of the file, an `EOFException` is raised.
+
+*Important* Forward seeks are implemented by reading and discarding the
+contents of the stream. The bigger the forward seek, the more data is thrown
+away, the longer the operation takes. And, if the data is being paid for over
+a long-haul S3 connection. the more the seek costs.
+
+Calling `seek()` on a `SelectInputStream` should only be used with care.
+
+The feature has been implemented for splittable queries across Selected data,
+where the initial read starts with a `seek()` to the offset. However, for
+reasons covered below, a codec should be used to declare the input unsplittable.
+
+## Use with third-party S3-compatible object stores.
+
+Third party object stores do not, at the time of writing, support S3 Select.
+S3 Select operations against such stores will fail, presumably with a store-specific
+error code.
+
+To avoid problems, disable S3 Select entirely:
+
+```xml
+<property>
+  <name>fs.s3a.select.enabled</name>
+  <value>false</value>
+</property>
+```
+
+This guarantees that the `hasCapability()` check will fail immediately,
+rather than delaying the failure until an SQL query is attempted.
+
+## Selecting data from the command line: `hadoop s3guard select`
+
+The `s3guard select` command allows direct select statements to be made
+of a path.
+
+Usage:
+
+```bash
+hadoop s3guard select [OPTIONS] \
+ [-limit rows] \
+ [-header (use|none|ignore)] \
+ [-out file] \
+ [-compression (gzip|none)] \
+ [-expected rows]
+ [-inputformat csv]
+ [-outputformat csv]
+  <PATH> <SELECT QUERY>
+```
+
+The output is printed, followed by some summary statistics, unless the `-out`
+option is used to declare a destination file. In this mode
+status will be logged to the console, but the output of the query will be
+saved directly to the output file.
+
+### Example 1
+
+Read the first 100 rows of the landsat dataset where cloud cover is zero:
+
+```bash
+hadoop s3guard select -header use -compression gzip -limit 100  \
+  s3a://landsat-pds/scene_list.gz \
+  "SELECT * FROM S3OBJECT s WHERE s.cloudCover = '0.0'"
+```
+
+### Example 2
+
+Return the `entityId` column for all rows in the dataset where the cloud
+cover was "0.0", and save it to the file `output.csv`:
+
+```bash
+hadoop s3guard select -header use -out s3a://mybucket/output.csv \
+  -compression gzip \
+  s3a://landsat-pds/scene_list.gz \
+  "SELECT s.entityId from S3OBJECT s WHERE s.cloudCover = '0.0'"
+```
+
+This file will:
+
+1. Be UTF-8 encoded.
+1. Have quotes on all columns returned.
+1. Use commas as a separator.
+1. Not have any header.
+
+The output can be saved to a file with the `-out` option. Note also that
+`-D key=value` settings can be used to control the operation, if placed after
+the `s3guard` command and before `select`
+
+
+```bash
+hadoop s3guard \
+  -D  s.s3a.select.output.csv.quote.fields=asneeded \
+  select \
+  -header use \
+  -compression gzip \
+  -limit 500 \
+  -inputformat csv \
+  -outputformat csv \
+  -out s3a://hwdev-steve-new/output.csv \
+  s3a://landsat-pds/scene_list.gz \
+  "SELECT s.entityId from S3OBJECT s WHERE s.cloudCover = '0.0'"
+```
+
+
+## Use in MR/Analytics queries: Work in Progress
+
+S3 Select support in analytics queries is a work in progress. It does
+not work reliably with large source files where the work is split up.
+
+As a proof of concept *only*, S3 Select queries can be made through
+MapReduce jobs which use any Hadoop `RecordReader`
+class which uses the new `openFile()` API.
+
+Currently this consists of the following MRv2 readers.
+
+```
+org.apache.hadoop.mapreduce.lib.input.LineRecordReader
+org.apache.hadoop.mapreduce.lib.input.FixedLengthRecordReader
+```
+
+And a limited number of the MRv1 record readers:
+
+```
+org.apache.hadoop.mapred.LineRecordReader
+```
+
+All of these readers use the new API and can be have its optional/mandatory
+options set via the `JobConf` used when creating/configuring the reader.
+
+These readers are instantiated within input formats; the following
+formats therefore support S3 Select.
+
+```
+org.apache.hadoop.mapreduce.lib.input.FixedLengthInputFormat
+org.apache.hadoop.mapreduce.lib.input.KeyValueTextInputFormat
+org.apache.hadoop.mapreduce.lib.input.NLineInputFormat
+org.apache.hadoop.mapreduce.lib.input.TextInputFormat
+org.apache.hadoop.mapred.KeyValueTextInputFormat
+org.apache.hadoop.mapred.TextInputFormat
+org.apache.hadoop.mapred.lib.NLineInputFormat
+```
+
+All `JobConf` options which begin with the prefix `mapreduce.job.input.file.option.`
+will have that prefix stripped and the remainder used as the name for an option
+when opening the file.
+
+All `JobConf` options which being with the prefix `mapreduce.job.input.file.must.`
+will be converted into mandatory options.
+
+To use an S3 Select call, set the following options
+
+```
+mapreduce.job.input.file.must.fs.s3a.select.sql = <SQL STATEMENT>
+mapreduce.job.input.file.must.fs.s3a.select.input.format = CSV
+mapreduce.job.input.file.must.fs.s3a.select.output.format = CSV
+```
+
+Further options may be set to tune the behaviour, for example:
+
+```java
+jobConf.set("mapreduce.job.input.file.must.fs.s3a.select.input.csv.header", "use");
+```
+
+*Note* How to tell if a reader has migrated to the new `openFile()` builder
+API:
+
+Set a mandatory option which is not known; if the job does not fail then
+an old reader is being used.
+
+```java
+jobConf.set("mapreduce.job.input.file.must.unknown.option", "anything");
+```
+
+
+### Querying Compressed objects
+
+S3 Select queries can be made against gzipped source files; the S3A input
+stream receives the output in text format, rather than as a (re)compressed
+stream.
+
+To read a gzip file, set `fs.s3a.select.input.compression` to `gzip`.
+
+```java
+jobConf.set("mapreduce.job.input.file.must.fs.s3a.select.input.compression",
+  "gzip");
+```
+
+
+Most of the Hadoop RecordReader classes automatically choose a decompressor
+based on the extension of the source file. This causes problems when
+reading `.gz` files, because S3 Select is automatically decompressing and
+returning csv-formatted text.
+
+By default, a query across gzipped files will fail with the error
+"IOException: not a gzip file"
+
+To avoid this problem, declare that the job should switch to the
+"Passthrough Codec" for all files with a ".gz" extension:
+
+```java
+jobConf.set("io.compression.codecs",
+    "org.apache.hadoop.io.compress.PassthroughCodec");
+jobConf.set("io.compress.passthrough.extension", ".gz");
+```
+
+Obviously, this breaks normal `.gz` decompression: only set it on S3 Select
+jobs.
+
+## S3 Select configuration options.
+
+Consult the javadocs for `org.apache.hadoop.fs.s3a.select.SelectConstants`.
+
+The listed options can be set in `core-site.xml`, supported by S3A per-bucket
+configuration, and can be set programmatically on the `Configuration` object
+use to configure a new filesystem instance.
+
+Any of these options can be set in the builder returned by the `openFile()` call
+—simply set them through a chain of `builder.must()` operations.
+
+```xml
+<property>
+  <name>fs.s3a.select.input.format</name>
+  <value>csv</value>
+  <description>Input format</description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.format</name>
+  <value>csv</value>
+  <description>Output format</description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.comment.marker</name>
+  <value>#</value>
+  <description>In S3 Select queries: the marker for comment lines in CSV files</description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.record.delimiter</name>
+  <value>\n</value>
+  <description>In S3 Select queries over CSV files: the record delimiter.
+    \t is remapped to the TAB character, \r to CR \n to newline. \\ to \
+    and \" to "
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.field.delimiter</name>
+  <value>,</value>
+  <description>In S3 Select queries over CSV files: the field delimiter.
+    \t is remapped to the TAB character, \r to CR \n to newline. \\ to \
+    and \" to "
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.quote.character</name>
+  <value>"</value>
+  <description>In S3 Select queries over CSV files: quote character.
+    \t is remapped to the TAB character, \r to CR \n to newline. \\ to \
+    and \" to "
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.quote.escape.character</name>
+  <value>\\</value>
+  <description>In S3 Select queries over CSV files: quote escape character.
+    \t is remapped to the TAB character, \r to CR \n to newline. \\ to \
+    and \" to "
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.csv.header</name>
+  <value>none</value>
+  <description>In S3 Select queries over CSV files: what is the role of the header? One of "none", "ignore" and "use"</description>
+</property>
+
+<property>
+  <name>fs.s3a.select.input.compression</name>
+  <value>none</value>
+  <description>In S3 Select queries, the source compression
+    algorithm. One of: "none" and "gzip"</description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.csv.quote.fields</name>
+  <value>always</value>
+  <description>
+    In S3 Select queries: should fields in generated CSV Files be quoted?
+    One of: "always", "asneeded".
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.csv.quote.character</name>
+  <value>"</value>
+  <description>
+    In S3 Select queries: the quote character for generated CSV Files.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.csv.quote.escape.character</name>
+  <value>\\</value>
+  <description>
+    In S3 Select queries: the quote escape character for generated CSV Files.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.csv.record.delimiter</name>
+  <value>\n</value>
+  <description>
+    In S3 Select queries: the record delimiter for generated CSV Files.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.output.csv.field.delimiter</name>
+  <value>,</value>
+  <description>
+    In S3 Select queries: the field delimiter for generated CSV Files.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.select.errors.include.sql</name>
+  <value>false</value>
+  <description>
+    Include the SQL statement in errors: this is useful for development but
+    may leak security and Personally Identifying Information in production,
+    so must be disabled there.
+  </description>
+</property>
+```
+
+## Security and Privacy
+
+SQL Injection attacks are the classic attack on data.
+Because S3 Select is a read-only API, the classic ["Bobby Tables"](https://xkcd.com/327/)
+attack to gain write access isn't going to work. Even so: sanitize your inputs.
+
+CSV does have security issues of its own, specifically:
+
+*Excel and other spreadsheets may interpret some fields beginning with special
+characters as formula, and execute them*
+
+S3 Select does not appear vulnerable to this, but in workflows where untrusted
+data eventually ends up in a spreadsheet (including Google Document spreadsheets),
+the data should be sanitized/audited first. There is no support for
+such sanitization in S3 Select or in the S3A connector.
+
+Logging Select statements may expose secrets if they are in the statement.
+Even if they are just logged, this may potentially leak Personally Identifying
+Information as covered in the EU GDPR legislation and equivalents.
+
+For both privacy and security reasons, SQL statements are not included
+in exception strings by default, nor logged at INFO level.
+
+To enable them, set `fs.s3a.select.errors.include.sql` to `true`, either in the
+site/application configuration, or as an option in the builder for a
+single request. When set, the request will also be logged at
+the INFO level of the log `org.apache.hadoop.fs.s3a.select.SelectBinding`.
+
+Personal Identifiable Information is not printed in the AWS S3 logs.
+Those logs contain only the SQL keywords from the query planner.
+All column names and literals are masked. Following is a sample log example:
+
+*Query:*
+
+```sql
+SELECT * FROM S3OBJECT s;
+```
+
+*Log:*
+
+```sql
+select (project (list (project_all))) (from (as str0 (id str1 case_insensitive)))
+```
+
+Note also that:
+
+1. Debug-level Hadoop logs for the module `org.apache.hadoop.fs.s3a` and other
+components's debug logs may also log the SQL statements (e.g. aws-sdk HTTP logs).
+
+The best practise here is: only enable SQL in exceptions while developing
+SQL queries, especially in an application/notebook where the exception
+text is a lot easier to see than the application logs.
+
+In production: don't log or report. If you do, all logs and output must be
+considered sensitive from security and privacy perspectives.
+
+The `hadoop s3guard select` command does enable the logging, so
+can be used as an initial place to experiment with the SQL syntax.
+Rationale: if you are constructing SQL queries on the command line,
+your shell history is already tainted with the query.
+
+### Links
+
+* [CVE-2014-3524](https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2014-3524).
+* [The Absurdly Underestimated Dangers of CSV Injection](http://georgemauer.net/2017/10/07/csv-injection.html).
+* [Comma Separated Vulnerabilities](https://www.contextis.com/blog/comma-separated-vulnerabilities).
+
+### SQL Syntax
+
+The SQL Syntax directly supported by the AWS S3 Select API is [documented by
+Amazon](https://docs.aws.amazon.com/AmazonS3/latest/dev/s3-glacier-select-sql-reference.html).
+
+* Use single quotes for all constants, not double quotes.
+* All CSV column values are strings unless cast to a type
+* Simple `SELECT` calls, no `JOIN`.
+
+### CSV formats
+
+"CSV" is less a format, more "a term meaning the data is in some nonstandard
+line-by-line" text file, and there are even "multiline CSV files".
+
+S3 Select only supports a subset of the loose "CSV" concept, as covered in
+the AWS documentation. There are also limits on how many columns and how
+large a single line may be.
+
+The specific quotation character, field and record delimiters, comments and escape
+characters can be configured in the Hadoop configuration.
+
+### Consistency, Concurrency and Error handling
+
+**Consistency**
+
+* Assume the usual S3 consistency model applies.
+
+* When enabled, S3Guard's DynamoDB table will declare whether or not
+a newly deleted file is visible: if it is marked as deleted, the
+select request will be rejected with a `FileNotFoundException`.
+
+* When an existing S3-hosted object is changed, the S3 select operation
+may return the results of a SELECT call as applied to either the old
+or new version.
+
+* We don't know whether you can get partially consistent reads, or whether
+an extended read ever picks up a later value.
+
+* The AWS S3 load balancers can briefly cache 404/Not-Found entries
+from a failed HEAD/GET request against a nonexistent file; this cached
+entry can briefly create create inconsistency, despite the
+AWS "Create is consistent" model. There is no attempt to detect or recover from
+this.
+
+**Concurrency**
+
+The outcome of what happens when source file is overwritten while the result of
+a select call is overwritten is undefined.
+
+The input stream returned by the operation is *NOT THREAD SAFE*.
+
+**Error Handling**
+
+If an attempt to issue an S3 select call fails, the S3A connector will
+reissue the request if-and-only-if it believes a retry may succeed.
+That is: it considers the operation to be idempotent and if the failure is
+considered to be a recoverable connectivity problem or a server-side rejection
+which can be retried (500, 503).
+
+If an attempt to read data from an S3 select stream (`org.apache.hadoop.fs.s3a.select.SelectInputStream)` fails partway through the read, *no attempt is made to retry the operation*
+
+In contrast, the normal S3A input stream tries to recover from (possibly transient)
+failures by attempting to reopen the file.
+
+
+## Performance
+
+The select operation is best when the least amount of data is returned by
+the query, as this reduces the amount of data downloaded.
+
+* Limit the number of columns projected to only those needed.
+* Use `LIMIT` to set an upper limit on the rows read, rather than implementing
+a row counter in application code and closing the stream when reached.
+This avoids having to abort the HTTPS connection and negotiate a new one
+on the next S3 request.
+
+The select call itself can be slow, especially when the source is a multi-MB
+compressed file with aggressive filtering in the `WHERE` clause.
+Assumption: the select query starts at row 1 and scans through each row,
+and does not return data until it has matched one or more rows.
+
+If the asynchronous nature of the `openFile().build().get()` sequence
+can be taken advantage of, by performing other work before or in parallel
+to the `get()` call: do it.
+
+## Troubleshooting
+
+Getting S3 Select code to work is hard, though those knowledgeable in SQL
+will find it easier.
+
+Problems can be split into:
+
+1. Basic configuration of the client to issue the query.
+1. Bad SQL select syntax and grammar.
+1. Datatype casting issues
+1. Bad records/data in source files.
+1. Failure to configure MR jobs to work correctly.
+1. Failure of MR jobs due to
+
+The exceptions here are all based on the experience during writing tests;
+more may surface with broader use.
+
+All failures other than network errors on request initialization are considered
+unrecoverable and will not be reattempted.
+
+As parse-time errors always state the line and column of an error, you can
+simplify debugging by breaking a SQL statement across lines, e.g.
+
+```java
+String sql = "SELECT\n"
+    + "s.entityId \n"
+    + "FROM " + "S3OBJECT s WHERE\n"
+    + "s.\"cloudCover\" = '100.0'\n"
+    + " LIMIT 100";
+```
+Now if the error is declared as "line 4", it will be on the select conditions;
+the column offset will begin from the first character on that row.
+
+The SQL Statements issued are only included in exceptions if `fs.s3a.select.errors.include.sql`
+is explicitly set to true. This can be done in an application during development,
+or in a `openFile()` option parameter. This should only be done during development,
+to reduce the risk of logging security or privacy information.
+
+
+### "mid-query" failures on large datasets
+
+S3 Select returns paged results; the source file is _not_ filtered in
+one go in the initial request.
+
+This means that errors related to the content of the data (type casting, etc)
+may only surface partway through the read. The errors reported in such a
+case may be different than those raised on reading the first page of data,
+where it will happen earlier on in the read process.
+
+### External Resources on for troubleshooting
+
+See:
+
+* [SELECT Command Reference](https://docs.aws.amazon.com/AmazonS3/latest/dev/s3-glacier-select-sql-reference-select.html)
+* [SELECT Object Content](https://docs.aws.amazon.com/AmazonS3/latest/API/RESTObjectSELECTContent.html)
+
+### IOException: "not a gzip file"
+
+This surfaces when trying to read in data from a `.gz` source file through an MR
+or other analytics query, and the gzip codec has tried to parse it.
+
+```
+java.io.IOException: not a gzip file
+at org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor.processBasicHeader(BuiltInGzipDecompressor.java:496)
+at org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor.executeHeaderState(BuiltInGzipDecompressor.java:257)
+at org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor.decompress(BuiltInGzipDecompressor.java:186)
+at org.apache.hadoop.io.compress.DecompressorStream.decompress(DecompressorStream.java:111)
+at org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:105)
+at java.io.InputStream.read(InputStream.java:101)
+at org.apache.hadoop.util.LineReader.fillBuffer(LineReader.java:182)
+at org.apache.hadoop.util.LineReader.readCustomLine(LineReader.java:306)
+at org.apache.hadoop.util.LineReader.readLine(LineReader.java:174)
+at org.apache.hadoop.mapreduce.lib.input.LineRecordReader.skipUtfByteOrderMark(LineRecordReader.java:158)
+at org.apache.hadoop.mapreduce.lib.input.LineRecordReader.nextKeyValue(LineRecordReader.java:198)
+```
+
+The underlying problem is that the gzip decompressor is automatically enabled
+when the the source file ends with the ".gz" extension. Because S3 Select
+returns decompressed data, the codec fails.
+
+The workaround here is to declare that the job should add the "Passthrough Codec"
+to its list of known decompressors, and that this codec should declare the
+file format it supports to be ".gz".
+
+```
+io.compression.codecs = org.apache.hadoop.io.compress.PassthroughCodec
+io.compress.passthrough.extension = .gz
+```
+
+### AWSBadRequestException `InvalidColumnIndex`
+
+
+Your SQL is wrong and the element at fault is considered an unknown column
+name.
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException:
+  Select: SELECT * FROM S3OBJECT WHERE odd = true on test/testSelectOddLines.csv:
+  com.amazonaws.services.s3.model.AmazonS3Exception:
+  The column index at line 1, column 30 is invalid.
+  Please check the service documentation and try again.
+  (Service: Amazon S3; Status Code: 400; Error Code: InvalidColumnIndex;
+```
+
+Here it's the first line of the query, column 30. Paste the query
+into an editor and position yourself on the line and column at fault.
+
+```sql
+SELECT * FROM S3OBJECT WHERE odd = true
+                             ^ HERE
+```
+
+Another example:
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException: Select:
+SELECT * FROM S3OBJECT s WHERE s._1 = "true" on test/testSelectOddLines.csv:
+  com.amazonaws.services.s3.model.AmazonS3Exception:
+  The column index at line 1, column 39 is invalid.
+  Please check the service documentation and try again.
+  (Service: Amazon S3; Status Code: 400;
+  Error Code: InvalidColumnIndex;
+```
+
+Here it is because strings must be single quoted, not double quoted.
+
+```sql
+SELECT * FROM S3OBJECT s WHERE s._1 = "true"
+                                      ^ HERE
+```
+
+S3 select uses double quotes to wrap column names, interprets the string
+as column "true", and fails with a non-intuitive message.
+
+*Tip*: look for the element at fault and treat the `InvalidColumnIndex`
+message as a parse-time message, rather than the definitive root
+cause of the problem.
+
+### AWSBadRequestException `ParseInvalidPathComponent`
+
+Your SQL is wrong.
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException:
+Select: SELECT * FROM S3OBJECT s WHERE s.'odd' is "true" on test/testSelectOddLines.csv
+: com.amazonaws.services.s3.model.AmazonS3Exception: Invalid Path component,
+  expecting either an IDENTIFIER or STAR, got: LITERAL,at line 1, column 34.
+  (Service: Amazon S3; Status Code: 400; Error Code: ParseInvalidPathComponent;
+
+```
+
+```
+SELECT * FROM S3OBJECT s WHERE s.'odd' is "true" on test/testSelectOddLines.csv
+                                 ^ HERE
+```
+
+
+### AWSBadRequestException  `ParseExpectedTypeName`
+
+Your SQL is still wrong.
+
+```
+
+org.apache.hadoop.fs.s3a.AWSBadRequestException:
+ Select: SELECT * FROM S3OBJECT s WHERE s.odd = "true"
+on test/testSelectOddLines.csv:
+com.amazonaws.services.s3.model.AmazonS3Exception
+: Expected type name, found QUOTED_IDENTIFIER:'true' at line 1, column 41.
+(Service: Amazon S3; Status Code: 400; Error Code: ParseExpectedTypeName;
+```
+
+### `ParseUnexpectedToken`
+
+Your SQL is broken.
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException:
+Select: SELECT * FROM S3OBJECT s WHERE s.5 = `true` on test/testSelectOddLines.csv:
+com.amazonaws.services.s3.model.AmazonS3Exception:
+Unexpected token found LITERAL:5d-1 at line 1, column 33.
+(Service: Amazon S3; Status Code: 400; Error Code: ParseUnexpectedToken;
+```
+### `ParseUnexpectedOperator`
+
+Your SQL is broken.
+
+```
+com.amazonaws.services.s3.model.AmazonS3Exception: Unexpected operator OPERATOR:'%' at line 1, column 45.
+(Service: Amazon S3; Status Code: 400;
+Error Code: ParseUnexpectedOperator; Request ID: E87F30C57436B459;
+S3 Extended Request ID: UBFOIgkQxBBL+bcBFPaZaPBsjdnd8NRz3NFWAgcctqm3n6f7ib9FMOpR+Eu1Cy6cNMYHCpJbYEY
+ =:ParseUnexpectedOperator: Unexpected operator OPERATOR:'%' at line 1, column 45.
+at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
+at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
+```
+
+### `MissingHeaders`
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException:
+Select: SELECT * FROM S3OBJECT s WHERE s."odd" = `true` on test/testSelectOddLines.csv:
+com.amazonaws.services.s3.model.AmazonS3Exception:
+Some headers in the query are missing from the file.
+Please check the file and try again.
+(Service: Amazon S3; Status Code: 400; Error Code: MissingHeaders;
+```
+
+1. There's a header used in the query which doesn't match any in the document
+itself.
+1. The header option for the select query is set to "none" or "ignore", and
+you are trying to use a header named there.
+
+This can happen if you are trying to use double quotes for constants in the
+SQL expression.
+
+```
+SELECT * FROM S3OBJECT s WHERE s."odd" = "true" on test/testSelectOddLines.csv:
+                                         ^ HERE
+```
+
+Double quotes (") may only be used when naming columns; for constants
+single quotes are required.
+
+### Method not allowed
+
+```
+org.apache.hadoop.fs.s3a.AWSS3IOException: Select on test/testSelectWholeFile:
+com.amazonaws.services.s3.model.AmazonS3Exception: The specified method is not
+allowed against this resource. (Service: Amazon S3; Status Code: 405;
+Error Code: MethodNotAllowed;
+```
+
+You are trying to use S3 Select to read data which for some reason
+you are not allowed to.
+
+### AWSBadRequestException `InvalidTextEncoding`
+
+The file couldn't be parsed. This can happen if you try to read a `.gz` file
+and forget to set the compression in the select request.
+
+That can be done through the `fs.s3a.select.compression` option.
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException:
+  Select: '" SELECT * FROM S3OBJECT s WHERE endstation_name = 'Bayswater Road: Hyde Park' "
+  on s3a://example/dataset.csv.gz:
+  com.amazonaws.services.s3.model.AmazonS3Exception:
+   UTF-8 encoding is required. The text encoding error was found near byte 8,192.
+    (Service: Amazon S3; Status Code: 400; Error Code: InvalidTextEncoding
+```
+
+### AWSBadRequestException  `InvalidCompressionFormat` "GZIP is not applicable to the queried object"
+
+A SELECT call has been made using a compression which doesn't match that of the
+source object, such as it being a plain text file.
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException: Select:
+ '" SELECT * FROM S3OBJECT s WHERE endstation_name = 'Bayswater Road: Hyde Park' "
+  on s3a://example/dataset.csv:
+   com.amazonaws.services.s3.model.AmazonS3Exception:
+    GZIP is not applicable to the queried object. Please correct the request and try again.
+     (Service: Amazon S3; Status Code: 400; Error Code: InvalidCompressionFormat;
+  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:212)
+  at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
+...
+Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: GZIP is not applicable to the queried object.
+ Please correct the request and try again.
+  Service: Amazon S3; Status Code: 400; Error Code: InvalidCompressionFormat;
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse
+  ...
+```
+
+### `PathIOException`: "seek() not supported"
+
+The input stream returned by the select call does not support seeking
+backwards in the stream.
+
+Similarly, `PositionedReadable` operations will fail when used to read
+data any offset other than that of `getPos()`.
+
+```
+org.apache.hadoop.fs.PathIOException: `s3a://landsat-pds/landsat.csv.gz': seek() not supported
+
+  at org.apache.hadoop.fs.s3a.select.SelectInputStream.unsupported(SelectInputStream.java:254)
+  at org.apache.hadoop.fs.s3a.select.SelectInputStream.seek(SelectInputStream.java:243)
+  at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:66)
+```
+
+There is no fix for this. You can move forward in a file using `skip(offset)`;
+bear in mind that the return value indicates what offset was skipped -it
+may be less than expected.
+
+### `IllegalArgumentException`: "Unknown mandatory key "fs.s3a.select.sql"
+
+The filesystem is not an S3A filesystem, and the s3a select option is not recognized.
+
+```
+java.lang.IllegalArgumentException: Unknown mandatory key "fs.s3a.select.sql"
+at com.google.common.base.Preconditions.checkArgument(Preconditions.java:88)
+at org.apache.hadoop.fs.AbstractFSBuilder.lambda$rejectUnknownMandatoryKeys$0(AbstractFSBuilder.java:331)
+at java.lang.Iterable.forEach(Iterable.java:75)
+at java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080)
+at org.apache.hadoop.fs.AbstractFSBuilder.rejectUnknownMandatoryKeys(AbstractFSBuilder.java:330)
+at org.apache.hadoop.fs.filesystem.openFileWithOptions(FileSystem.java:3541)
+at org.apache.hadoop.fs.FileSystem$FSDataInputStreamBuilder.build(FileSystem.java:4442)
+```
+
+* Verify that the URL has an "s3a:" prefix.
+* If it does, there may be a non-standard S3A implementation, or some
+a filtering/relaying class has been placed in front of the S3AFilesystem.
+
+### `IllegalArgumentException`: "Unknown mandatory key in non-select file I/O"
+
+The file options to tune an S3 select call are only valid when a SQL expression
+is set in the `fs.s3a.select.sql` option. If not, any such option added as a `must()` value
+will fail.
+
+```
+java.lang.IllegalArgumentException: Unknown mandatory key for s3a://example/test/testSelectOptionsOnlyOnSelectCalls.csv in non-select file I/O "fs.s3a.select.input.csv.header"
+
+  at com.google.common.base.Preconditions.checkArgument(Preconditions.java:115)
+  at org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.lambda$rejectUnknownMandatoryKeys$0(AbstractFSBuilderImpl.java:352)
+  at java.lang.Iterable.forEach(Iterable.java:75)
+  at java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080)
+  at org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(AbstractFSBuilderImpl.java:351)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.openFileWithOptions(S3AFileSystem.java:3736)
+  at org.apache.hadoop.fs.FileSystem$FSDataInputStreamBuilder.build(FileSystem.java:4471)
+```
+
+Requiring these options without providing a SQL query is invariably an error.
+Fix: add the SQL statement, or use `opt()` calls to set the option.
+
+If the `fs.s3a.select.sql` option is set, and still a key is rejected, then
+either the spelling of the key is wrong, it has leading or trailing spaces,
+or it is an option not supported in that specific release of Hadoop.
+
+
+### PathIOException : "seek() backwards from  not supported"
+
+Backwards seeks in an S3 Select `SelectInputStream` are not supported.
+
+```
+org.apache.hadoop.fs.PathIOException: `s3a://landsat-pds/scene_list.gz':
+  seek() backwards from 16387 to 0 not supported
+
+  at org.apache.hadoop.fs.s3a.select.SelectInputStream.unsupported(SelectInputStream.java:288)
+  at org.apache.hadoop.fs.s3a.select.SelectInputStream.seek(SelectInputStream.java:253)
+  at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:66)
+```
+
+### InvalidTableAlias
+
+The SELECT refers to the name of a column which is not recognized
+
+* the name of a column is wrong, here `s.oddf`.
+* headers are not enabled for the CSV source file. Fix: enable.
+* a generated alias is used e.g `s._1`, but headers have been enabled.
+Fix. disable, or use the header name.
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException:
+ SELECT * FROM S3OBJECT WHERE s."oddf" = 'true'
+ on s3a://example/test/testParseBrokenCSVFile:
+ com.amazonaws.services.s3.model.AmazonS3Exception:
+ Invalid table alias is specified at line 1, column 30.
+  Please check the file and try again. (Service: Amazon S3; Status Code: 400; Error Code: InvalidTableAlias;
+   Invalid table alias is specified at line 1, column 30. Please check the file and try again.
+    (Service: Amazon S3; Status Code: 400;
+    Error Code: InvalidTableAlias;
+    Request ID: 8693B86A52CFB91C;
+  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:225)
+  at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
+  at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265)
+  ...
+Caused by: com.amazonaws.services.s3.model.AmazonS3Exception:
+ Invalid table alias is specified at line 1, column 30.
+  Please check the file and try again.
+   (Service: Amazon S3; Status Code: 400; Error Code: InvalidTableAlias; Request ID: 8693B86A52CFB91C;
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1640)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1304)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1058)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
+```
+
+###  `AWSBadRequestException` "Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP."
+
+A string field could not be converted to a timestamp because one or more of its entries were not parseable
+with the given timestamp.
+
+Example, from a spreadsheet where "timestamp" is normally a well-formatted timestamp field,
+but in one column it is just "Tuesday"
+
+```sql
+SELECT CAST(s.date AS TIMESTAMP) FROM S3OBJECT s
+```
+
+```
+org.apache.hadoop.fs.s3a.AWSBadRequestException: Select on s3a://example/test/testParseBrokenCSVFile:
+com.amazonaws.services.s3.model.AmazonS3Exception:
+Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP.
+(Service: Amazon S3; Status Code: 400; Error Code: CastFailed;
+Request ID: E2158FE45AF2049A; S3 Extended Request ID: iM40fzGuaPt6mQo0QxDDX+AY1bAgSVD1sKErFq6Y4GDJYHIAnmc00i0EvGGnH+0MFCFhKIivIrQ=),
+S3 Extended Request ID: iM40fzGuaPt6mQo0QxDDX+AY1bAgSVD1sKErFq6Y4GDJYHIAnmc00i0EvGGnH+0MFCFhKIivIrQ=:CastFailed:
+Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP.
+(Service: Amazon S3; Status Code: 400; Error Code: CastFailed; Request ID: E2158FE45AF2049A; S3 Extended Request ID: iM40fzGuaPt6mQo0QxDDX+AY1bAgSVD1sKErFq6Y4GDJYHIAnmc00i0EvGGnH+0MFCFhKIivIrQ=)
+  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:225)
+  at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
+  at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265)
+Caused by: com.amazonaws.services.s3.model.AmazonS3Exception:
+ Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP.
+  (Service: Amazon S3; Status Code: 400; Error Code: CastFailed;)
+
+```
+
+There's no way to recover from a bad record here; no option to skip invalid
+rows.
+
+*Note:* This is an example stack trace *without* the SQL being printed.
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
index 34ba029..068269c 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
@@ -478,6 +478,22 @@ the `fs.s3a.scale.test.csvfile` option set to its path.
 (yes, the space is necessary. The Hadoop `Configuration` class treats an empty
 value as "do not override the default").
 
+### Turning off S3 Select
+
+The S3 select tests are skipped when the S3 endpoint doesn't support S3 Select.
+
+```xml
+<property>
+  <name>fs.s3a.select.enabled</name>
+  <value>false</value>
+</property>
+```
+
+If your endpoint doesn't support that feature, this option should be in
+your `core-site.xml` file, so that trying to use S3 select fails fast with
+a meaningful error ("S3 Select not supported") rather than a generic Bad Request
+exception.
+
 
 ### Testing Session Credentials
 
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
index 267646c..9e8a871 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
@@ -39,6 +39,7 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.getCSVTestPath;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.junit.Assert.*;
 
@@ -150,8 +151,7 @@ public class ITestS3AAWSCredentialsProvider {
     Configuration conf = new Configuration();
     conf.set(AWS_CREDENTIALS_PROVIDER,
         AnonymousAWSCredentialsProvider.class.getName());
-    Path testFile = new Path(
-        conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
+    Path testFile = getCSVTestPath(conf);
     FileSystem fs = FileSystem.newInstance(testFile.toUri(), conf);
     assertNotNull(fs);
     assertTrue(fs instanceof S3AFileSystem);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
index 5cd7379..5c2b5a3 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import org.junit.Assume;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -37,6 +36,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVPath;
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 
 /**
@@ -152,12 +152,9 @@ public class ITestS3AFailureHandling extends AbstractS3ATestBase {
 
   @Test
   public void testMultiObjectDeleteNoPermissions() throws Throwable {
-    Configuration conf = getConfiguration();
-    String csvFile = conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
-    Assume.assumeTrue("CSV test file is not the default",
-        DEFAULT_CSVTEST_FILE.equals(csvFile));
-    Path testFile = new Path(csvFile);
-    S3AFileSystem fs = (S3AFileSystem)testFile.getFileSystem(conf);
+    Path testFile = getLandsatCSVPath(getConfiguration());
+    S3AFileSystem fs = (S3AFileSystem)testFile.getFileSystem(
+        getConfiguration());
     intercept(MultiObjectDeleteException.class,
         () -> removeKeys(fs, fs.pathToKey(testFile)));
   }
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 484f079..e15c24a 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
@@ -47,7 +47,6 @@ import com.amazonaws.auth.AWSCredentialsProvider;
 import org.hamcrest.core.Is;
 import org.junit.Assert;
 import org.junit.Assume;
-import org.junit.internal.AssumptionViolatedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -63,6 +62,7 @@ import java.util.Map;
 import java.util.concurrent.Callable;
 
 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.s3a.FailureInjectionPolicy.*;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
@@ -144,7 +144,6 @@ public final class S3ATestUtils {
    * @param purge flag to enable Multipart purging
    * @return the FS
    * @throws IOException IO Problems
-   * @throws AssumptionViolatedException if the FS is not named
    */
   public static S3AFileSystem createTestFileSystem(Configuration conf,
       boolean purge)
@@ -158,12 +157,10 @@ public final class S3ATestUtils {
       testURI = URI.create(fsname);
       liveTest = testURI.getScheme().equals(Constants.FS_S3A);
     }
-    if (!liveTest) {
-      // This doesn't work with our JUnit 3 style test cases, so instead we'll
-      // make this whole class not run by default
-      throw new AssumptionViolatedException(
-          "No test filesystem in " + TEST_FS_S3A_NAME);
-    }
+    // This doesn't work with our JUnit 3 style test cases, so instead we'll
+    // make this whole class not run by default
+    Assume.assumeTrue("No test filesystem in " + TEST_FS_S3A_NAME,
+        liveTest);
     // patch in S3Guard options
     maybeEnableS3Guard(conf);
     S3AFileSystem fs1 = new S3AFileSystem();
@@ -192,7 +189,6 @@ public final class S3ATestUtils {
    * @param conf configuration
    * @return the FS
    * @throws IOException IO Problems
-   * @throws AssumptionViolatedException if the FS is not named
    */
   public static FileContext createTestFileContext(Configuration conf)
       throws IOException {
@@ -204,12 +200,10 @@ public final class S3ATestUtils {
       testURI = URI.create(fsname);
       liveTest = testURI.getScheme().equals(Constants.FS_S3A);
     }
-    if (!liveTest) {
-      // This doesn't work with our JUnit 3 style test cases, so instead we'll
-      // make this whole class not run by default
-      throw new AssumptionViolatedException("No test filesystem in "
-          + TEST_FS_S3A_NAME);
-    }
+    // This doesn't work with our JUnit 3 style test cases, so instead we'll
+    // make this whole class not run by default
+    Assume.assumeTrue("No test filesystem in " + TEST_FS_S3A_NAME,
+        liveTest);
     // patch in S3Guard options
     maybeEnableS3Guard(conf);
     FileContext fc = FileContext.getFileContext(testURI, conf);
@@ -327,11 +321,57 @@ public final class S3ATestUtils {
       String defVal) {
     String confVal = conf != null ? conf.getTrimmed(key, defVal) : defVal;
     String propval = System.getProperty(key);
-    return StringUtils.isNotEmpty(propval) && !UNSET_PROPERTY.equals(propval)
+    return isNotEmpty(propval) && !UNSET_PROPERTY.equals(propval)
         ? propval : confVal;
   }
 
   /**
+   * Get the test CSV file; assume() that it is not empty.
+   * @param conf test configuration
+   * @return test file.
+   */
+  public static String getCSVTestFile(Configuration conf) {
+    String csvFile = conf
+        .getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
+    Assume.assumeTrue("CSV test file is not the default",
+        isNotEmpty(csvFile));
+    return csvFile;
+  }
+
+  /**
+   * Get the test CSV path; assume() that it is not empty.
+   * @param conf test configuration
+   * @return test file as a path.
+   */
+  public static Path getCSVTestPath(Configuration conf) {
+    return new Path(getCSVTestFile(conf));
+  }
+
+  /**
+   * Get the test CSV file; assume() that it is not modified (i.e. we haven't
+   * switched to a new storage infrastructure where the bucket is no longer
+   * read only).
+   * @return test file.
+   * @param conf test configuration
+   */
+  public static String getLandsatCSVFile(Configuration conf) {
+    String csvFile = getCSVTestFile(conf);
+    Assume.assumeTrue("CSV test file is not the default",
+        DEFAULT_CSVTEST_FILE.equals(csvFile));
+    return csvFile;
+  }
+  /**
+   * Get the test CSV file; assume() that it is not modified (i.e. we haven't
+   * switched to a new storage infrastructure where the bucket is no longer
+   * read only).
+   * @param conf test configuration
+   * @return test file as a path.
+   */
+  public static Path getLandsatCSVPath(Configuration conf) {
+    return new Path(getLandsatCSVFile(conf));
+  }
+
+  /**
    * Verify the class of an exception. If it is not as expected, rethrow it.
    * Comparison is on the exact class, not subclass-of inference as
    * offered by {@code instanceof}.
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
index e8467e7..3822ee7 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
@@ -101,8 +101,7 @@ public class TestS3AAWSCredentialsProvider {
         TemporaryAWSCredentialsProvider.NAME
             + ", \t" + SimpleAWSCredentialsProvider.NAME
             + " ,\n " + AnonymousAWSCredentialsProvider.NAME);
-    Path testFile = new Path(
-        conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
+    Path testFile = getCSVTestPath(conf);
 
     AWSCredentialProviderList list = createAWSCredentialProviderSet(
         testFile.toUri(), conf);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
index 246bf9d..69a6ed6 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
@@ -178,7 +178,7 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase {
    * @return fork ID string in a format parseable by Jobs
    * @throws Exception failure
    */
-  protected String randomJobId() throws Exception {
+  public static String randomJobId() throws Exception {
     String testUniqueForkId = System.getProperty(TEST_UNIQUE_FORK_ID, "0001");
     int l = testUniqueForkId.length();
     String trailingDigits = testUniqueForkId.substring(l - 4, l);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
index e10c85b..71e9975 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
@@ -24,7 +24,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
-import java.io.PrintStream;
 import java.net.URI;
 import java.util.Collection;
 import java.util.HashSet;
@@ -37,7 +36,6 @@ import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.util.StopWatch;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.FileSystem;
-import org.junit.Assume;
 import org.junit.Test;
 
 import org.apache.hadoop.conf.Configuration;
@@ -64,6 +62,7 @@ import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_BAD_STATE;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_NO_METASTORE_OR_FILESYSTEM;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_USAGE;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
@@ -88,11 +87,21 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     assertEquals(message, expected, tool.run(args));
   }
 
-  protected static void expectSuccess(
+  /**
+   * Expect a command to succeed.
+   * @param message any extra text to include in the assertion error message
+   * @param tool tool to run
+   * @param args arguments to the command
+   * @return the output of any successful run
+   * @throws Exception failure
+   */
+  protected static String expectSuccess(
       String message,
       S3GuardTool tool,
       String... args) throws Exception {
-    assertEquals(message, SUCCESS, tool.run(args));
+    ByteArrayOutputStream buf = new ByteArrayOutputStream();
+    exec(SUCCESS, message, tool, buf, args);
+    return buf.toString();
   }
 
   /**
@@ -450,58 +459,6 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
         () -> run(S3GuardTool.Init.NAME));
   }
 
-  /**
-   * Get the test CSV file; assume() that it is not modified (i.e. we haven't
-   * switched to a new storage infrastructure where the bucket is no longer
-   * read only).
-   * @return test file.
-   */
-  protected String getLandsatCSVFile() {
-    String csvFile = getConfiguration()
-        .getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
-    Assume.assumeTrue("CSV test file is not the default",
-        DEFAULT_CSVTEST_FILE.equals(csvFile));
-    return csvFile;
-  }
-
-  /**
-   * Execute a command, returning the buffer if the command actually completes.
-   * If an exception is raised the output is logged instead.
-   * @param cmd command
-   * @param args argument list
-   * @throws Exception on any failure
-   */
-  public String exec(S3GuardTool cmd, String...args) throws Exception {
-    ByteArrayOutputStream buf = new ByteArrayOutputStream();
-    try {
-      exec(cmd, buf, args);
-      return buf.toString();
-    } catch (AssertionError e) {
-      throw e;
-    } catch (Exception e) {
-      LOG.error("Command {} failed: \n{}", cmd, buf);
-      throw e;
-    }
-  }
-
-  /**
-   * Execute a command, saving the output into the buffer.
-   * @param cmd command
-   * @param buf buffer to use for tool output (not SLF4J output)
-   * @param args argument list
-   * @throws Exception on any failure
-   */
-  protected void exec(S3GuardTool cmd, ByteArrayOutputStream buf, String...args)
-      throws Exception {
-    LOG.info("exec {}", (Object) args);
-    int r = 0;
-    try(PrintStream out =new PrintStream(buf)) {
-      r = cmd.run(args, out);
-      out.flush();
-    }
-    assertEquals("Command " + cmd + " failed\n"+ buf, 0, r);
-  }
-
   @Test
   public void
   testDiffCommand() throws Exception {
@@ -537,7 +494,7 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     ByteArrayOutputStream buf = new ByteArrayOutputStream();
     S3GuardTool.Diff cmd = new S3GuardTool.Diff(fs.getConf());
     cmd.setStore(ms);
-    exec(cmd, buf, "diff", "-meta", DYNAMODB_TABLE, testPath.toString());
+    exec(0, "", cmd, buf, "diff", "-meta", DYNAMODB_TABLE, testPath.toString());
 
     Set<Path> actualOnS3 = new HashSet<>();
     Set<Path> actualOnMS = new HashSet<>();
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
index 97173fe..aa88b0b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
@@ -49,7 +49,7 @@ import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_NAME_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_TAG;
 import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.*;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
-import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec;
 
 /**
  * Test S3Guard related CLI commands against DynamoDB.
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
index 1ee3cde..6a4d45e 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
@@ -40,7 +40,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 
 import static org.apache.hadoop.fs.s3a.MultipartTestUtils.*;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVFile;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
@@ -97,7 +99,7 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
   public void testDestroyBucketExistsButNoTable() throws Throwable {
     run(Destroy.NAME,
         "-meta", LOCAL_METADATA,
-        getLandsatCSVFile());
+        getLandsatCSVFile(getConfiguration()));
   }
 
   @Test
@@ -161,7 +163,7 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
   public void testLandsatBucketUnguarded() throws Throwable {
     run(BucketInfo.NAME,
         "-" + BucketInfo.UNGUARDED_FLAG,
-        getLandsatCSVFile());
+        getLandsatCSVFile(getConfiguration()));
   }
 
   @Test
@@ -169,14 +171,15 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
     runToFailure(E_BAD_STATE,
         BucketInfo.NAME,
         "-" + BucketInfo.GUARDED_FLAG,
-        ITestS3GuardToolLocal.this.getLandsatCSVFile());
+        getLandsatCSVFile(
+            ITestS3GuardToolLocal.this.getConfiguration()));
   }
 
   @Test
   public void testLandsatBucketRequireUnencrypted() throws Throwable {
     run(BucketInfo.NAME,
         "-" + BucketInfo.ENCRYPTION_FLAG, "none",
-        getLandsatCSVFile());
+        getLandsatCSVFile(getConfiguration()));
   }
 
   @Test
@@ -184,7 +187,8 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
     runToFailure(E_BAD_STATE,
         BucketInfo.NAME,
         "-" + BucketInfo.ENCRYPTION_FLAG,
-        "AES256", ITestS3GuardToolLocal.this.getLandsatCSVFile());
+        "AES256", getLandsatCSVFile(
+            ITestS3GuardToolLocal.this.getConfiguration()));
   }
 
   @Test
@@ -367,7 +371,7 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
       allOptions.add(String.valueOf(ageSeconds));
     }
     allOptions.add(path.toString());
-    exec(cmd, buf, allOptions.toArray(new String[0]));
+    exec(0, "", cmd, buf, allOptions.toArray(new String[0]));
 
     try (BufferedReader reader = new BufferedReader(
         new InputStreamReader(new ByteArrayInputStream(buf.toByteArray())))) {
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardToolTestHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardToolTestHelper.java
new file mode 100644
index 0000000..f22aa36
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardToolTestHelper.java
@@ -0,0 +1,89 @@
+/*
+ * 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.s3guard;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Helper class for tests which make CLI invocations of the S3Guard tools.
+ * That's {@link AbstractS3GuardToolTestBase} and others.
+ */
+public final class S3GuardToolTestHelper {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      S3GuardToolTestHelper.class);
+
+  private S3GuardToolTestHelper() {
+  }
+
+  /**
+   * Execute a command, returning the buffer if the command actually completes.
+   * If an exception is raised the output is logged instead.
+   * @param cmd command
+   * @param args argument list
+   * @throws Exception on any failure
+   */
+  public static String exec(S3GuardTool cmd, String... args) throws Exception {
+    ByteArrayOutputStream buf = new ByteArrayOutputStream();
+    try {
+      exec(0, "", cmd, buf, args);
+      return buf.toString();
+    } catch (AssertionError e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Command {} failed: \n{}", cmd, buf);
+      throw e;
+    }
+  }
+
+  /**
+   * Execute a command, saving the output into the buffer.
+   * @param expectedResult expected result of the command.
+   * @param errorText error text to include in the assertion.
+   * @param cmd command
+   * @param buf buffer to use for tool output (not SLF4J output)
+   * @param args argument list
+   * @throws Exception on any failure
+   */
+  public static void exec(final int expectedResult,
+      final String errorText,
+      final S3GuardTool cmd,
+      final ByteArrayOutputStream buf,
+      final String... args)
+      throws Exception {
+    LOG.info("exec {}", (Object) args);
+    int r;
+    try (PrintStream out = new PrintStream(buf)) {
+      r = cmd.run(args, out);
+      out.flush();
+    }
+    if (expectedResult != r) {
+      String message = errorText.isEmpty() ? "" : (errorText + ": ")
+          + "Command " + cmd + " failed\n" + buf;
+      assertEquals(message, expectedResult, r);
+    }
+  }
+
+}
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
new file mode 100644
index 0000000..18138a6
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java
@@ -0,0 +1,746 @@
+/*
+ * 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.select;
+
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.time.Duration;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.Scanner;
+import java.util.function.Consumer;
+
+import org.junit.Assume;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileContext;
+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.StreamCapabilities;
+import org.apache.hadoop.fs.s3a.AWSServiceIOException;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.commit.AbstractCommitITest;
+import org.apache.hadoop.fs.s3a.commit.DurationInfo;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.PassthroughCodec;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.lib.input.LineRecordReader;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+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;
+
+/**
+ * Superclass for S3 Select tests.
+ * A lot of the work here goes into creating and querying a simple CSV test
+ * format, with various datatypes which can be used in type-casting queries.
+ * <pre>
+ * 1  "ID": index of the row
+ * 2  "date": date as ISO 8601
+ * 3  "timestamp": timestamp in seconds of epoch
+ * 4  "name", entry-$row
+ * 5  "odd", odd/even as boolean. True means odd,
+ * 6  "oddint", odd/even as int : 1 for odd, 0 for even
+ * 7  "oddrange": odd/even as 1 for odd, -1 for even
+ * </pre>
+ */
+public abstract class AbstractS3SelectTest extends AbstractS3ATestBase {
+
+  /**
+   * Number of columns in the CSV file: {@value}.
+   */
+  public static final int CSV_COLUMN_COUNT = 7;
+
+  protected static final String TRUE = q("TRUE");
+
+  protected static final String FALSE = q("FALSE");
+
+  public static final String SELECT_EVERYTHING = "SELECT * FROM S3OBJECT s";
+
+  public static final String SELECT_EVEN_ROWS_NO_HEADER =
+      "SELECT * FROM S3OBJECT s WHERE s._5 = " + TRUE;
+  public static final String SELECT_ODD_ROWS
+      = "SELECT s.name FROM S3OBJECT s WHERE s.odd = " + TRUE;
+
+  public static final String SELECT_ODD_ENTRIES
+      = "SELECT * FROM S3OBJECT s WHERE s.odd = `TRUE`";
+
+  public static final String SELECT_ODD_ENTRIES_BOOL
+      = "SELECT * FROM S3OBJECT s WHERE CAST(s.odd AS BOOL) = TRUE";
+
+  public static final String SELECT_ODD_ENTRIES_INT
+      = "SELECT * FROM S3OBJECT s WHERE CAST(s.\"oddint\" AS INT) = 1";
+
+  public static final String SELECT_ODD_ENTRIES_DECIMAL
+      = "SELECT * FROM S3OBJECT s WHERE CAST(s.\"oddint\" AS DECIMAL) = 1";
+
+  /**
+   * Playing with timestamps: {@value}.
+   */
+  public static final String SELECT_TO_DATE
+      = "SELECT\n"
+      + "CAST(s.\"date\" AS TIMESTAMP)\n"
+      + "FROM S3OBJECT s";
+
+
+  /**
+   * How many rows are being generated.
+   */
+  protected static final int ALL_ROWS_COUNT = 10;
+
+  /**
+   * Row count of all rows + header.
+   */
+  protected static final int ALL_ROWS_COUNT_WITH_HEADER = ALL_ROWS_COUNT + 1;
+
+  /**
+   * Number of odd rows expected: {@value}.
+   */
+  protected static final int ODD_ROWS_COUNT = ALL_ROWS_COUNT / 2;
+
+  /**
+   * Number of even rows expected: {@value}.
+   * This is the same as the odd row count; it's separate just to
+   * be consistent on tests which select even results.
+   */
+  protected static final int EVEN_ROWS_COUNT = ODD_ROWS_COUNT;
+
+  protected static final String ENTRY_0001 = "\"entry-0001\"";
+
+  protected static final String ENTRY_0002 = "\"entry-0002\"";
+
+  /**
+   * Path to the landsat csv.gz file.
+   */
+  private Path landsatGZ;
+
+  /**
+   * The filesystem with the landsat data.
+   */
+  private S3AFileSystem landsatFS;
+
+
+  // A random task attempt id for testing.
+  private String attempt0;
+
+  private TaskAttemptID taskAttempt0;
+
+  private String jobId;
+
+  /**
+   * Base CSV file is headers.
+   * <pre>
+   * 1  "ID": index of the row
+   * 2  "date": date as Date.toString
+   * 3  "timestamp": timestamp in seconds of epoch
+   * 4  "name", entry-$row
+   * 5  "odd", odd/even as boolean
+   * 6  "oddint", odd/even as int : 1 for odd, 0 for even
+   * 7  "oddrange": odd/even as 1 for odd, -1 for even
+   * </pre>
+   * @param fs filesystem
+   * @param path path to write
+   * @param header should the standard header be printed?
+   * @param quoteHeaderPolicy what the header quote policy is.
+   * @param quoteRowPolicy what the row quote policy is.
+   * @param rows number of rows
+   * @param separator column separator
+   * @param eol end of line characters
+   * @param quote quote char
+   * @param footer callback to run after the main CSV file is written
+   * @throws IOException IO failure.
+   */
+  public static void createStandardCsvFile(
+      final FileSystem fs,
+      final Path path,
+      final boolean header,
+      final long quoteHeaderPolicy,
+      final long quoteRowPolicy,
+      final int rows,
+      final String separator,
+      final String eol,
+      final String quote,
+      final Consumer<CsvFile> footer) throws IOException {
+    try (CsvFile csv = new CsvFile(fs,
+        path,
+        true,
+        separator,
+        eol,
+        quote)) {
+
+      if (header) {
+        writeStandardHeader(csv, quoteHeaderPolicy);
+      }
+      DateTimeFormatter formatter
+          = DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+      ZonedDateTime timestamp = ZonedDateTime.now();
+      Duration duration = Duration.ofHours(20);
+      // loop is at 1 for use in counters and flags
+      for (int i = 1; i <= rows; i++) {
+        // flip the odd flags
+        boolean odd = (i & 1) == 1;
+        // and move the timestamp back
+        timestamp = timestamp.minus(duration);
+        csv.row(quoteRowPolicy,
+            i,
+            timestamp.format(formatter),
+            timestamp.toEpochSecond(),
+            String.format("entry-%04d", i),
+            odd ? "TRUE" : "FALSE",
+            odd ? 1 : 0,
+            odd ? 1 : -1
+        );
+      }
+      // write the footer
+      footer.accept(csv);
+    }
+  }
+
+  /**
+   * Write out the standard header to a CSV file.
+   * @param csv CSV file to use.
+   * @param quoteHeaderPolicy quote policy.
+   * @return the input file.
+   * @throws IOException failure to write.
+   */
+  private static CsvFile writeStandardHeader(final CsvFile csv,
+      final long quoteHeaderPolicy) throws IOException {
+    return csv.row(quoteHeaderPolicy,
+        "id",
+        "date",
+        "timestamp",
+        "name",
+        "odd",
+        "oddint",
+        "oddrange");
+  }
+
+  /**
+   * Verify that an exception has a specific error code.
+   * if not: an assertion is raised containing the original value.
+   * @param code expected code.
+   * @param ex exception caught
+   * @throws AssertionError on a mismatch
+   */
+  protected static AWSServiceIOException verifyErrorCode(final String code,
+      final AWSServiceIOException ex) {
+    logIntercepted(ex);
+    if (!code.equals(ex.getErrorCode())) {
+      throw new AssertionError("Expected Error code" + code
+          + " actual " + ex.getErrorCode(),
+          ex);
+    }
+    return ex;
+  }
+
+  /**
+   * Probe for a filesystem instance supporting S3 Select.
+   * @param filesystem filesystem
+   * @return true iff the filesystem supports S3 Select.
+   */
+  boolean isSelectAvailable(final FileSystem filesystem) {
+    return filesystem instanceof StreamCapabilities
+        && ((StreamCapabilities) filesystem)
+        .hasCapability(S3_SELECT_CAPABILITY);
+  }
+
+  /**
+   * Setup: requires select to be available.
+   */
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    Assume.assumeTrue("S3 Select is not enabled on "
+            + getFileSystem().getUri(),
+        isSelectAvailable(getFileSystem()));
+    Configuration conf = getConfiguration();
+    landsatGZ = getLandsatCSVPath(conf);
+    landsatFS = (S3AFileSystem) landsatGZ.getFileSystem(conf);
+    Assume.assumeTrue("S3 Select is not enabled on " + landsatFS.getUri(),
+        isSelectAvailable(landsatFS));
+    // create some job info
+    jobId = AbstractCommitITest.randomJobId();
+    attempt0 = "attempt_" + jobId + "_m_000000_0";
+    taskAttempt0 = TaskAttemptID.forName(attempt0);
+  }
+
+  /**
+   * Build the SQL statement, using String.Format rules.
+   * @param template template
+   * @param args arguments for the template
+   * @return the template to use
+   */
+  protected static String sql(
+      final String template,
+      final Object... args) {
+    return args.length > 0 ? String.format(template, args) : template;
+  }
+
+  /**
+   * Quote a constant with the SQL quote logic.
+   * @param c constant
+   * @return quoted constant
+   */
+  protected static String q(String c) {
+    return '\'' + c + '\'';
+  }
+
+  /**
+   * Select from a source file.
+   * @param fileSystem FS.
+   * @param source source file.
+   * @param conf config for the select call.
+   * @param sql template for a formatted SQL request.
+   * @param args arguments for the formatted request.
+   * @return the input stream.
+   * @throws IOException failure
+   */
+  protected FSDataInputStream select(
+      final FileSystem fileSystem,
+      final Path source,
+      final Configuration conf,
+      final String sql,
+      final Object... args)
+      throws IOException {
+    String expression = sql(sql, args);
+    describe("Execution Select call: %s", expression);
+    FutureDataInputStreamBuilder builder =
+        fileSystem.openFile(source)
+            .must(SELECT_SQL, expression);
+    // propagate all known options
+    for (String key : InternalSelectConstants.SELECT_OPTIONS) {
+      String value = conf.get(key);
+      if (value != null) {
+        builder.must(key, value);
+      }
+    }
+    return awaitFuture(builder.build());
+  }
+
+  /**
+   * Select from a source file via the file context API.
+   * @param fc file context
+   * @param source source file.
+   * @param conf config for the select call.
+   * @param sql template for a formatted SQL request.
+   * @param args arguments for the formatted request.
+   * @return the input stream.
+   * @throws IOException failure
+   */
+  protected FSDataInputStream select(
+      final FileContext fc,
+      final Path source,
+      final Configuration conf,
+      final String sql,
+      final Object... args)
+      throws IOException {
+    String expression = sql(sql, args);
+    describe("Execution Select call: %s", expression);
+    FutureDataInputStreamBuilder builder = fc.openFile(source)
+        .must(SELECT_SQL, expression);
+    // propagate all known options
+    InternalSelectConstants.SELECT_OPTIONS.forEach((key) ->
+        Optional.ofNullable(conf.get(key))
+            .map((v) -> builder.must(key, v)));
+    return awaitFuture(builder.build());
+  }
+
+  /**
+   * Parse a selection to lines; log at info.
+   * @param selection selection input
+   * @return a list of lines.
+   * @throws IOException if raised during the read.
+   */
+  protected List<String> parseToLines(final FSDataInputStream selection)
+      throws IOException {
+    return parseToLines(selection, getMaxLines());
+  }
+
+  /**
+   * Enable the passthrough codec for a job, with the given extension.
+   * @param conf configuration to update
+   * @param extension extension to use
+   */
+  protected void enablePassthroughCodec(final Configuration conf,
+      final String extension) {
+    conf.set(CommonConfigurationKeys.IO_COMPRESSION_CODECS_KEY,
+        PassthroughCodec.CLASSNAME);
+    conf.set(PassthroughCodec.OPT_EXTENSION, extension);
+  }
+
+  /**
+   * Override if a test suite is likely to ever return more lines.
+   * @return the max number for parseToLines/1
+   */
+  protected int getMaxLines() {
+    return 100;
+  }
+
+  /**
+   * Parse a selection to lines; log at info.
+   * @param selection selection input
+   * @param maxLines maximum number of lines.
+   * @return a list of lines.
+   * @throws IOException if raised during the read.
+   */
+  protected List<String> parseToLines(final FSDataInputStream selection,
+      int maxLines)
+      throws IOException {
+    List<String> result = new ArrayList<>();
+    String stats;
+    // the scanner assumes that any IOE => EOF; we don't want
+    // that and so will check afterwards.
+    try (Scanner scanner = new Scanner(
+        new BufferedReader(new InputStreamReader(selection)))) {
+      scanner.useDelimiter(CSV_INPUT_RECORD_DELIMITER_DEFAULT);
+      while (maxLines > 0) {
+        try {
+          String l = scanner.nextLine();
+          LOG.info("{}", l);
+          result.add(l);
+          maxLines--;
+        } catch (NoSuchElementException e) {
+          // EOL or an error
+          break;
+        }
+      }
+      stats = selection.toString();
+      describe("Result line count: %s\nStatistics\n%s",
+          result.size(), stats);
+      // look for any raised error.
+      IOException ioe = scanner.ioException();
+      if (ioe != null && !(ioe instanceof EOFException)) {
+        throw ioe;
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Verify the selection count; return the original list.
+   * If there's a mismatch, the whole list is logged at error, then
+   * an assertion raised.
+   * @param expected expected value.
+   * @param expression expression -for error messages.
+   * @param selection selected result.
+   * @return the input list.
+   */
+  protected List<String> verifySelectionCount(
+      final int expected,
+      final String expression,
+      final List<String> selection) {
+    return verifySelectionCount(expected, expected, expression, selection);
+  }
+
+  /**
+   * Verify the selection count is within a given range;
+   * return the original list.
+   * If there's a mismatch, the whole list is logged at error, then
+   * an assertion raised.
+   * @param min min value (exclusive).
+   * @param max max value (exclusive). If -1: no maximum.
+   * @param expression expression -for error messages.
+   * @param selection selected result.
+   * @return the input list.
+   */
+  protected List<String> verifySelectionCount(
+      final int min,
+      final int max,
+      final String expression,
+      final List<String> selection) {
+    int size = selection.size();
+    if (size < min || (max > -1 && size > max)) {
+      // mismatch: log and then fail
+      String listing = prepareToPrint(selection);
+      LOG.error("\n{} => \n{}", expression, listing);
+      fail("row count from select call " + expression
+          + " is out of range " + min + " to " + max
+          + ": " + size
+          + " \n" + listing);
+    }
+    return selection;
+  }
+
+  /**
+   * Do whatever is needed to prepare a string for logging.
+   * @param selection selection
+   * @return something printable.
+   */
+  protected String prepareToPrint(final List<String> selection) {
+    return String.join("\n", selection);
+  }
+
+  /**
+   * Create "the standard" CSV file with the default row count.
+   * @param fs filesystem
+   * @param path path to write
+   * @param quoteRowPolicy what the row quote policy is.
+   * @throws IOException IO failure.
+   */
+  protected void createStandardCsvFile(
+      final FileSystem fs,
+      final Path path,
+      final long quoteRowPolicy)
+      throws IOException {
+    createStandardCsvFile(
+        fs, path,
+        true,
+        ALL_QUOTES,
+        quoteRowPolicy,
+        ALL_ROWS_COUNT,
+        ",",
+        "\n",
+        "\"",
+        c -> {});
+  }
+
+  /**
+   * Set an MR Job input option.
+   * @param conf configuration
+   * @param key key to set
+   * @param val value
+   */
+  void inputOpt(Configuration conf, String key, String val) {
+    conf.set(MRJobConfig.INPUT_FILE_OPTION_PREFIX + key, val);
+  }
+
+  /**
+   * Set a mandatory MR Job input option.
+   * @param conf configuration
+   * @param key key to set
+   * @param val value
+   */
+  void inputMust(Configuration conf, String key, String val) {
+    conf.set(MRJobConfig.INPUT_FILE_MANDATORY_PREFIX + key,
+        val);
+  }
+
+  /**
+   * Reads lines through a v2 RecordReader, as if it were part of a
+   * MRv2 job.
+   * @param conf job conf
+   * @param path path to query
+   * @param sql sql to add to the configuration.
+   * @param initialCapacity capacity of the read
+   * @param reader reader: this is closed after the read
+   * @return the selected lines.
+   * @throws Exception failure
+   */
+  protected List<String> readRecords(JobConf conf,
+      Path path,
+      String sql,
+      RecordReader<?, ?> reader,
+      int initialCapacity) throws Exception {
+
+    inputMust(conf, SELECT_SQL, sql);
+    List<String> lines = new ArrayList<>(initialCapacity);
+    try {
+      reader.initialize(
+          createSplit(conf, path),
+          createTaskAttemptContext(conf));
+      while (reader.nextKeyValue()) {
+        lines.add(reader.getCurrentValue().toString());
+      }
+    } finally {
+      reader.close();
+    }
+    return lines;
+  }
+  /**
+   * Reads lines through a v1 RecordReader, as if it were part of a
+   * MRv1 job.
+   * @param conf job conf
+   * @param reader reader: this is closed after the read
+   * @param initialCapacity capacity of the read
+   * @return the selected lines.
+   * @throws Exception failure
+   */
+  protected <K, V> List<String> readRecordsV1(JobConf conf,
+      org.apache.hadoop.mapred.RecordReader<K, V> reader,
+      K key,
+      V value,
+      int initialCapacity) throws Exception {
+    List<String> lines = new ArrayList<>(initialCapacity);
+    try {
+      while (reader.next(key, value)) {
+        lines.add(value.toString());
+      }
+    } finally {
+      reader.close();
+    }
+    return lines;
+  }
+
+  /**
+   * Create a task attempt context for a job, creating a random JobID to
+   * do this.
+   * @param conf job configuration.
+   * @return a new task attempt context containing the job conf
+   * @throws Exception failure.
+   */
+  protected TaskAttemptContext createTaskAttemptContext(final JobConf conf)
+      throws Exception {
+    String id = AbstractCommitITest.randomJobId();
+    return new TaskAttemptContextImpl(conf,
+        TaskAttemptID.forName("attempt_" + id + "_m_000000_0"));
+  }
+
+  /**
+   * Create an MRv2 file input split.
+   * @param conf job configuration
+   * @param path path to file
+   * @return the split
+   * @throws IOException problems reading the file.
+   */
+  protected FileSplit createSplit(final JobConf conf, final Path path)
+      throws IOException {
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus status = fs.getFileStatus(path);
+    return new FileSplit(path, 0, status.getLen(),
+        new String[]{"localhost"});
+  }
+
+  /**
+   * Create an MRv1 file input split.
+   * @param conf job configuration
+   * @param path path to file
+   * @return the split
+   * @throws IOException problems reading the file.
+   */
+  protected org.apache.hadoop.mapred.FileSplit
+      createSplitV1(final JobConf conf, final Path path)
+      throws IOException {
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus status = fs.getFileStatus(path);
+    return new org.apache.hadoop.mapred.FileSplit(path, 0, status.getLen(),
+        new String[]{"localhost"});
+  }
+
+  /**
+   * Create a v2 line record reader expecting newlines as the EOL marker.
+   * @return a reader
+   */
+  protected RecordReader<LongWritable, Text> createLineRecordReader() {
+    return new LineRecordReader(new byte[]{'\n'});
+  }
+
+  /**
+   * Create a v1 line record reader.
+   * @return a reader
+   */
+  protected org.apache.hadoop.mapred.RecordReader<LongWritable, Text>
+      createLineRecordReaderV1(
+        final JobConf conf,
+        final Path path) throws IOException {
+    return new org.apache.hadoop.mapred.LineRecordReader(
+        conf, createSplitV1(conf, path));
+  }
+
+  /**
+   * Get the path to the landsat file.
+   * @return the landsat CSV.GZ path.
+   */
+  protected Path getLandsatGZ() {
+    return landsatGZ;
+  }
+
+  /**
+   * Get the filesystem for the landsat file.
+   * @return the landsat FS.
+   */
+  protected S3AFileSystem getLandsatFS() {
+    return landsatFS;
+  }
+
+  /**
+   * Perform a seek: log duration of the operation.
+   * @param stream stream to seek.
+   * @param target target position.
+   * @throws IOException on an error
+   */
+  protected void seek(final FSDataInputStream stream, final long target)
+      throws IOException {
+    try(DurationInfo ignored =
+            new DurationInfo(LOG, "Seek to %d", target)) {
+      stream.seek(target);
+    }
+  }
+
+  /**
+   * Execute a seek so far past the EOF that it will be rejected.
+   * If the seek did not fail, the exception raised includes the toString()
+   * value of the stream.
+   * @param seekStream stream to seek in.
+   * @param newpos new position
+   * @return the EOF Exception raised.
+   * @throws Exception any other exception.
+   */
+  protected EOFException expectSeekEOF(final FSDataInputStream seekStream,
+      final int newpos) throws Exception {
+    return intercept(EOFException.class,
+        () -> {
+          seek(seekStream, newpos);
+          // return this for the test failure reports.
+          return "Stream after seek to " + newpos + ": " + seekStream;
+        });
+  }
+
+  public String getAttempt0() {
+    return attempt0;
+  }
+
+  public TaskAttemptID getTaskAttempt0() {
+    return taskAttempt0;
+  }
+
+  public String getJobId() {
+    return jobId;
+  }
+
+  /**
+   * Logs intercepted exceptions.
+   * This generates the stack traces for the documentation.
+   * @param ex exception
+   * @return the exception passed in (for chaining)
+   */
+  protected static <T extends Exception> T logIntercepted(T ex) {
+    LOG.info("Intercepted Exception is ", ex);
+    return ex;
+  }
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/CsvFile.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/CsvFile.java
new file mode 100644
index 0000000..06e6d2a
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/CsvFile.java
@@ -0,0 +1,138 @@
+/*
+ * 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.select;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Writer for generating test CSV files.
+ *
+ * Quotes are manged by passing in a long whose specific bits control
+ * whether or not a row is quoted, bit 0 for column 0, etc.
+ */
+class CsvFile implements Closeable {
+
+
+  /** constant to quote all columns. */
+  public static final long ALL_QUOTES = 0x7fffffff;
+
+  /** quote nothing: {@value}. */
+  public static final long NO_QUOTES = 0;
+
+  private final Path path;
+
+  private final PrintWriter out;
+
+  private final String separator;
+
+  private final String eol;
+
+  private final String quote;
+
+  CsvFile(final FileSystem fs,
+      final Path path,
+      boolean overwrite,
+      final String separator,
+      final String eol,
+      final String quote) throws IOException {
+    this.path = path;
+    this.separator = Preconditions.checkNotNull(separator);
+    this.eol = Preconditions.checkNotNull(eol);
+    this.quote = Preconditions.checkNotNull(quote);
+    out = new PrintWriter(fs.create(path, overwrite));
+  }
+
+
+  /**
+   * Close the file, if not already done.
+   * @throws IOException on a failure.
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    if (out != null) {
+      out.close();
+    }
+  }
+
+  public Path getPath() {
+    return path;
+  }
+
+  public String getSeparator() {
+    return separator;
+  }
+
+  public String getEol() {
+    return eol;
+  }
+
+  /**
+   * Write a row.
+   * Entries are quoted if the bit for that column is true.
+   * @param quotes quote policy: every bit defines the rule for that element
+   * @param columns columns to write
+   * @return self for ease of chaining.
+   */
+  public CsvFile row(long quotes, Object... columns) {
+    for (int i = 0; i < columns.length; i++) {
+      if (i != 0) {
+        out.write(separator);
+      }
+      boolean toQuote = (quotes & 1) == 1;
+      // unsigned right shift to make next column flag @ position 0
+      quotes = quotes >>> 1;
+      if (toQuote) {
+        out.write(quote);
+      }
+      out.write(columns[i].toString());
+      if (toQuote) {
+        out.write(quote);
+      }
+    }
+    out.write(eol);
+    return this;
+  }
+
+  /**
+   * Write a line.
+   * @param line line to print
+   * @return self for ease of chaining.
+   * @throws IOException IO failure
+   */
+  public CsvFile line(String line) {
+    out.write(line);
+    out.write(eol);
+    return this;
+  }
+
+  /**
+   * Get the output stream.
+   * @return the stream.
+   */
+  public PrintWriter getOut() {
+    return out;
+  }
+}
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
new file mode 100644
index 0000000..5fe4e2b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java
@@ -0,0 +1,967 @@
+/*
+ * 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.select;
+
+import java.io.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import org.junit.Assume;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
+import org.apache.hadoop.fs.s3a.AWSBadRequestException;
+import org.apache.hadoop.fs.s3a.AWSServiceIOException;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3AInputStream;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.commit.DurationInfo;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.LineRecordReader;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+
+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.READAHEAD_RANGE;
+import static org.apache.hadoop.fs.s3a.select.CsvFile.ALL_QUOTES;
+import static org.apache.hadoop.fs.s3a.select.SelectBinding.expandBackslashChars;
+import static org.apache.hadoop.fs.s3a.select.SelectConstants.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.collection.IsCollectionWithSize.hasSize;
+
+/**
+ * Test the S3 Select feature with some basic SQL Commands.
+ * Executed if the destination store declares its support for the feature.
+ */
+public class ITestS3Select extends AbstractS3SelectTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestS3Select.class);
+
+  public static final String E_CAST_FAILED = "CastFailed";
+
+  public static final String E_PARSE_INVALID_PATH_COMPONENT
+      = "ParseInvalidPathComponent";
+
+  public static final String E_INVALID_TABLE_ALIAS = "InvalidTableAlias";
+
+  private Configuration selectConf;
+
+  /** well formed CSV. */
+  private Path csvPath;
+
+  /** CSV file with fewer columns than expected, all fields parse badly. */
+  private Path brokenCSV;
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    Assume.assumeTrue("S3 Select is not enabled",
+        getFileSystem().hasCapability(S3_SELECT_CAPABILITY));
+    csvPath = path(getMethodName() + ".csv");
+    selectConf = new Configuration(false);
+    selectConf.setBoolean(SELECT_ERRORS_INCLUDE_SQL, true);
+    createStandardCsvFile(getFileSystem(), csvPath, ALL_QUOTES);
+    // create the broken CSV file.
+    brokenCSV = path("testParseBrokenCSVFile");
+    createStandardCsvFile(
+        getFileSystem(), brokenCSV,
+        true,
+        ALL_QUOTES,
+        ALL_ROWS_COUNT,
+        ALL_ROWS_COUNT,
+        ",",
+        "\n",
+        "\"",
+        csv -> csv
+            .line("# comment")
+            .row(ALL_QUOTES, "bad", "Tuesday", 0, "entry-bad", "yes", false));
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    describe("teardown");
+    try {
+      if (csvPath != null) {
+        getFileSystem().delete(csvPath, false);
+      }
+      if (brokenCSV != null) {
+        getFileSystem().delete(brokenCSV, false);
+      }
+    } finally {
+      super.teardown();
+    }
+  }
+
+  @Test
+  public void testCapabilityProbe() throws Throwable {
+
+    // this should always hold true if we get past test setup
+    assertTrue("Select is not available on " + getFileSystem(),
+        isSelectAvailable(getFileSystem()));
+  }
+
+  @SuppressWarnings("NestedAssignment")
+  @Test
+  public void testReadWholeFileClassicAPI() throws Throwable {
+    describe("create and read the whole file. Verifies setup working");
+    int lines;
+    try (BufferedReader reader = new BufferedReader(
+        new InputStreamReader(
+            getFileSystem().open(csvPath)))) {
+      lines = 0;
+      // seek to 0, which is what some input formats do
+      String line;
+      while ((line = reader.readLine()) != null) {
+        lines++;
+        LOG.info("{}", line);
+      }
+    }
+    assertEquals("line count", ALL_ROWS_COUNT_WITH_HEADER, lines);
+  }
+
+  @Test
+  public void testSelectWholeFileNoHeader() throws Throwable {
+    describe("Select the entire file, expect all rows but the header");
+    expectSelected(
+        ALL_ROWS_COUNT,
+        selectConf,
+        CSV_HEADER_OPT_USE,
+        "SELECT * FROM S3OBJECT");
+  }
+
+  @Test
+  public void testSelectFirstColumnNoHeader() throws Throwable {
+    describe("Select the entire file, expect all rows but the header");
+    expectSelected(
+        ALL_ROWS_COUNT_WITH_HEADER,
+        selectConf,
+        CSV_HEADER_OPT_NONE,
+        "SELECT s._1 FROM S3OBJECT s");
+  }
+
+  @Test
+  public void testSelectSelfNoHeader() throws Throwable {
+    describe("Select the entire file, expect all rows but the header");
+    expectSelected(
+        ALL_ROWS_COUNT_WITH_HEADER,
+        selectConf,
+        CSV_HEADER_OPT_NONE,
+        "SELECT s._1 FROM S3OBJECT s WHERE s._1 = s._1");
+  }
+
+  @Test
+  public void testSelectSelfUseHeader() throws Throwable {
+    describe("Select the entire file, expect all rows including the header");
+    expectSelected(
+        ALL_ROWS_COUNT,
+        selectConf,
+        CSV_HEADER_OPT_USE,
+        "SELECT s.id FROM S3OBJECT s WHERE s.id = s.id");
+  }
+
+  @Test
+  public void testSelectID2UseHeader() throws Throwable {
+    describe("Select where ID=2; use the header");
+    expectSelected(
+        1,
+        selectConf,
+        CSV_HEADER_OPT_USE,
+        "SELECT s.id FROM S3OBJECT s WHERE s.id = '2'");
+  }
+
+  @Test
+  public void testSelectNoMatchingID() throws Throwable {
+    describe("Select where there is no match; expect nothing back");
+    expectSelected(
+        0,
+        selectConf,
+        CSV_HEADER_OPT_USE,
+        "SELECT s.id FROM S3OBJECT s WHERE s.id = '0x8000'");
+  }
+
+  @Test
+  public void testSelectId1() throws Throwable {
+    describe("Select the first element in the file");
+    expectSelected(
+        1,
+        selectConf,
+        CSV_HEADER_OPT_NONE,
+        "SELECT * FROM S3OBJECT s WHERE s._1 = '1'",
+        TRUE);
+  }
+
+  @Test
+  public void testSelectEmptySQL() throws Throwable {
+    describe("An empty SQL statement fails fast");
+    FutureDataInputStreamBuilder builder = getFileSystem().openFile(
+        csvPath)
+        .must(SELECT_SQL, "");
+    interceptFuture(IllegalArgumentException.class,
+        SELECT_SQL,
+        builder.build());
+  }
+
+  @Test
+  public void testSelectEmptyFile() throws Throwable {
+    describe("Select everything from an empty file");
+    Path path = path("testSelectEmptyFile");
+    S3AFileSystem fs = getFileSystem();
+    ContractTestUtils.touch(fs, path);
+    parseToLines(fs.openFile(path)
+            .must(SELECT_SQL, SELECT_EVERYTHING)
+            .build()
+            .get(),
+        0);
+  }
+
+  @Test
+  public void testSelectEmptyFileWithConditions() throws Throwable {
+    describe("Select everything from an empty file with a more complex SQL");
+    Path path = path("testSelectEmptyFileWithConditions");
+    S3AFileSystem fs = getFileSystem();
+    ContractTestUtils.touch(fs, path);
+    String sql = "SELECT * FROM S3OBJECT s WHERE s._1 = `TRUE`";
+    CompletableFuture<FSDataInputStream> future = fs.openFile(path)
+        .must(SELECT_SQL, sql).build();
+    assertEquals("Not at the end of the file", -1, future.get().read());
+  }
+
+  @Test
+  public void testSelectSeek() throws Throwable {
+    describe("Verify forward seeks work, not others");
+
+    // start: read in the full data through the initial select
+    // this makes asserting that contents match possible
+    Path path = csvPath;
+    S3AFileSystem fs = getFileSystem();
+    int len = (int) fs.getFileStatus(path).getLen();
+    byte[] fullData = new byte[len];
+    int actualLen;
+    try (DurationInfo ignored =
+             new DurationInfo(LOG, "Initial read of %s", path);
+        FSDataInputStream sourceStream =
+             select(fs, path,
+                 selectConf,
+                 SELECT_EVERYTHING)) {
+      // read it in
+      actualLen = IOUtils.read(sourceStream, fullData);
+    }
+    int seekRange = 20;
+
+    try (FSDataInputStream seekStream =
+             select(fs, path,
+                 selectConf,
+                 SELECT_EVERYTHING)) {
+      SelectInputStream sis
+          = (SelectInputStream) seekStream.getWrappedStream();
+      S3AInstrumentation.InputStreamStatistics streamStats
+          = sis.getS3AStreamStatistics();
+      // lazy seek doesn't raise a problem here
+      seekStream.seek(0);
+      assertEquals("first byte read", fullData[0], seekStream.read());
+
+      // and now the pos has moved, again, seek will be OK
+      seekStream.seek(1);
+      seekStream.seek(1);
+      // but trying to seek elsewhere now fails
+      PathIOException ex = intercept(PathIOException.class,
+          SelectInputStream.SEEK_UNSUPPORTED,
+          () -> seekStream.seek(0));
+      LOG.info("Seek error is as expected", ex);
+      // positioned reads from the current location work.
+      byte[] buffer = new byte[1];
+      long pos = seekStream.getPos();
+      seekStream.readFully(pos, buffer);
+      // but positioned backwards fail.
+      intercept(PathIOException.class,
+          SelectInputStream.SEEK_UNSUPPORTED,
+          () -> seekStream.readFully(0, buffer));
+      // the position has now moved on.
+      assertPosition(seekStream, pos + 1);
+      // so a seek to the old pos will fail
+      intercept(PathIOException.class,
+          SelectInputStream.SEEK_UNSUPPORTED,
+          () -> seekStream.readFully(pos, buffer));
+
+      // set the readahead to the default.
+      // This verifies it reverts to the default.
+      seekStream.setReadahead(null);
+      assertEquals("Readahead in ",
+          Constants.DEFAULT_READAHEAD_RANGE, sis.getReadahead());
+      // forward seeks are implemented as 1+ skip
+      long target = seekStream.getPos() + seekRange;
+      seek(seekStream, target);
+      assertPosition(seekStream, target);
+      // now do a read and compare values
+      assertEquals("byte at seek position",
+          fullData[(int)seekStream.getPos()], seekStream.read());
+      assertEquals("Seek bytes skipped in " + streamStats,
+          seekRange, streamStats.bytesSkippedOnSeek);
+
+      // try an invalid readahead range
+      intercept(IllegalArgumentException.class,
+          S3AInputStream.E_NEGATIVE_READAHEAD_VALUE,
+          () -> seekStream.setReadahead(-1L));
+
+      // do a slightly forward offset read
+      int read = seekStream.read(seekStream.getPos() + 2, buffer, 0, 1);
+      assertEquals(1, read);
+
+      // final fun: seek way past the EOF
+      logIntercepted(expectSeekEOF(seekStream, actualLen * 2));
+      assertPosition(seekStream, actualLen);
+      assertEquals(-1, seekStream.read());
+      LOG.info("Seek statistics {}", streamStats);
+      // this will return no, but not fail
+      assertFalse("Failed to seek to new source in " + seekStream,
+          seekStream.seekToNewSource(0));
+      // and set the readahead to 0 to see that close path works
+      seekStream.setReadahead(0L);
+      // then do a manual close even though there's one in the try resource.
+      // which will verify that a double close is harmless
+      seekStream.close();
+      LOG.info("Final stream state {}", sis);
+    }
+  }
+
+  /**
+   * Assert that a stream is in a specific position.
+   * @param stream stream or other seekable.
+   * @param pos expected position.
+   * @throws IOException failure of the getPos() call.
+   * @throws AssertionError mismatch between expected and actual.
+   */
+  private void assertPosition(Seekable stream, long pos)
+      throws IOException {
+    assertEquals("Wrong stream position in " + stream,
+        pos, stream.getPos());
+  }
+
+  @Test
+  public void testSelectOddLinesNoHeader() throws Throwable {
+    describe("Select odd lines, ignoring the header");
+    expectSelected(
+        ODD_ROWS_COUNT,
+        selectConf,
+        CSV_HEADER_OPT_IGNORE,
+        "SELECT * FROM S3OBJECT s WHERE s._5 = `TRUE`");
+    // and do a quick check on the instrumentation
+    long bytesRead = getFileSystem().getInstrumentation()
+        .getCounterValue(Statistic.STREAM_SEEK_BYTES_READ);
+    assertNotEquals("No bytes read count", 0, bytesRead);
+  }
+
+  @Test
+  public void testSelectOddLinesHeader() throws Throwable {
+    describe("Select the odd values");
+    List<String> selected = expectSelected(
+        ODD_ROWS_COUNT,
+        selectConf,
+        CSV_HEADER_OPT_USE,
+        SELECT_ODD_ROWS);
+    // the list includes odd values
+    assertThat(selected, hasItem(ENTRY_0001));
+    // but not the evens
+    assertThat(selected, not(hasItem(ENTRY_0002)));
+  }
+
+  @Test
+  public void testSelectOddLinesHeaderTSVOutput() throws Throwable {
+    describe("Select the odd values with tab spaced output");
+    selectConf.set(CSV_OUTPUT_FIELD_DELIMITER, "\t");
+    selectConf.set(CSV_OUTPUT_QUOTE_CHARACTER, "'");
+    selectConf.set(CSV_OUTPUT_QUOTE_FIELDS,
+        CSV_OUTPUT_QUOTE_FIELDS_AS_NEEEDED);
+    selectConf.set(CSV_OUTPUT_RECORD_DELIMITER, "\r");
+    List<String> selected = expectSelected(
+        ODD_ROWS_COUNT,
+        selectConf,
+        CSV_HEADER_OPT_USE,
+        SELECT_ODD_ENTRIES_BOOL);
+    // the list includes odd values
+    String row1 = selected.get(0);
+
+    // split that first line into columns: This is why TSV is better for code
+    // to work with than CSV
+    String[] columns = row1.split("\t", -1);
+    assertEquals("Wrong column count from tab split line <" + row1 + ">",
+        CSV_COLUMN_COUNT, columns.length);
+    assertEquals("Wrong column value from tab split line <" + row1 + ">",
+        "entry-0001", columns[3]);
+  }
+
+  @Test
+  public void testSelectNotOperationHeader() throws Throwable {
+    describe("Select the even values with a NOT call; quote the header name");
+    List<String> selected = expectSelected(
+        EVEN_ROWS_COUNT,
+        selectConf,
+        CSV_HEADER_OPT_USE,
+        "SELECT s.name FROM S3OBJECT s WHERE NOT s.\"odd\" = %s",
+        TRUE);
+    // the list includes no odd values
+    assertThat(selected, not(hasItem(ENTRY_0001)));
+    // but has the evens
+    assertThat(selected, hasItem(ENTRY_0002));
+  }
+
+  @Test
+  public void testBackslashExpansion() throws Throwable {
+    assertEquals("\t\r\n", expandBackslashChars("\t\r\n"));
+    assertEquals("\t", expandBackslashChars("\\t"));
+    assertEquals("\r", expandBackslashChars("\\r"));
+    assertEquals("\r \n", expandBackslashChars("\\r \\n"));
+    assertEquals("\\", expandBackslashChars("\\\\"));
+  }
+
+  /**
+   * This is an expanded example for the documentation.
+   * Also helps catch out unplanned changes to the configuration strings.
+   */
+  @Test
+  public void testSelectFileExample() throws Throwable {
+    describe("Select the entire file, expect all rows but the header");
+    int len = (int) getFileSystem().getFileStatus(csvPath).getLen();
+    FutureDataInputStreamBuilder builder =
+        getFileSystem().openFile(csvPath)
+            .must("fs.s3a.select.sql",
+                SELECT_ODD_ENTRIES)
+            .must("fs.s3a.select.input.format", "CSV")
+            .must("fs.s3a.select.input.compression", "NONE")
+            .must("fs.s3a.select.input.csv.header", "use")
+            .must("fs.s3a.select.output.format", "CSV");
+
+    CompletableFuture<FSDataInputStream> future = builder.build();
+    try (FSDataInputStream select = future.get()) {
+      // process the output
+      byte[] bytes = new byte[len];
+      int actual = select.read(bytes);
+      LOG.info("file length is {}; length of selected data is {}",
+          len, actual);
+    }
+  }
+
+  /**
+   * This is an expanded example for the documentation.
+   * Also helps catch out unplanned changes to the configuration strings.
+   */
+  @Test
+  public void testSelectUnsupportedInputFormat() throws Throwable {
+    describe("Request an unsupported input format");
+    FutureDataInputStreamBuilder builder = getFileSystem().openFile(csvPath)
+        .must(SELECT_SQL, SELECT_ODD_ENTRIES)
+        .must(SELECT_INPUT_FORMAT, "pptx");
+    interceptFuture(IllegalArgumentException.class,
+        "pptx",
+        builder.build());
+  }
+
+  /**
+   * Ask for an invalid output format.
+   */
+  @Test
+  public void testSelectUnsupportedOutputFormat() throws Throwable {
+    describe("Request a (currently) unsupported output format");
+    FutureDataInputStreamBuilder builder = getFileSystem().openFile(csvPath)
+        .must(SELECT_SQL, SELECT_ODD_ENTRIES)
+        .must(SELECT_INPUT_FORMAT, "csv")
+        .must(SELECT_OUTPUT_FORMAT, "json");
+    interceptFuture(IllegalArgumentException.class,
+        "json",
+        builder.build());
+  }
+
+  /**
+   *  Missing files fail lazy.
+   */
+  @Test
+  public void testSelectMissingFile() throws Throwable {
+
+    describe("Select a missing file, expect it to surface in the future");
+
+    Path missing = path("missing");
+
+    FutureDataInputStreamBuilder builder =
+        getFileSystem().openFile(missing)
+            .must(SELECT_SQL, SELECT_ODD_ENTRIES);
+
+    interceptFuture(FileNotFoundException.class,
+        "", builder.build());
+  }
+
+  @Test
+  public void testSelectDirectoryFails() throws Throwable {
+    describe("Verify that secondary select options are only valid on select"
+        + " queries");
+    S3AFileSystem fs = getFileSystem();
+    Path dir = path("dir");
+    // this will be an empty dir marker
+    fs.mkdirs(dir);
+
+    FutureDataInputStreamBuilder builder =
+        getFileSystem().openFile(dir)
+            .must(SELECT_SQL, SELECT_ODD_ENTRIES);
+    interceptFuture(PathIOException.class,
+        "", builder.build());
+
+    // try the parent
+    builder = getFileSystem().openFile(dir.getParent())
+            .must(SELECT_SQL,
+                SELECT_ODD_ENTRIES);
+    interceptFuture(PathIOException.class,
+        "", builder.build());
+  }
+
+  @Test
+  public void testSelectRootFails() throws Throwable {
+    describe("verify root dir selection is rejected");
+    FutureDataInputStreamBuilder builder =
+        getFileSystem().openFile(path("/"))
+            .must(SELECT_SQL, SELECT_ODD_ENTRIES);
+    interceptFuture(PathIOException.class,
+        "", builder.build());
+  }
+
+  /**
+   * Validate the abort logic.
+   */
+  @Test
+  public void testCloseWithAbort() throws Throwable {
+    describe("Close the stream with the readahead outstanding");
+    S3ATestUtils.MetricDiff readOps = new S3ATestUtils.MetricDiff(
+        getFileSystem(),
+        Statistic.STREAM_READ_OPERATIONS_INCOMPLETE);
+    selectConf.setInt(READAHEAD_RANGE, 2);
+
+    FSDataInputStream stream = select(getFileSystem(), csvPath, selectConf,
+        "SELECT * FROM S3OBJECT s");
+    SelectInputStream sis = (SelectInputStream) stream.getWrappedStream();
+    assertEquals("Readahead on " + sis, 2, sis.getReadahead());
+    stream.setReadahead(1L);
+    assertEquals("Readahead on " + sis, 1, sis.getReadahead());
+    stream.read();
+    S3AInstrumentation.InputStreamStatistics stats
+        = sis.getS3AStreamStatistics();
+    assertEquals("Read count in " + sis,
+        1, stats.bytesRead);
+    stream.close();
+    assertEquals("Abort count in " + sis,
+        1, stats.aborted);
+    readOps.assertDiffEquals("Read operations are still considered active",
+        0);
+    intercept(PathIOException.class, FSExceptionMessages.STREAM_IS_CLOSED,
+        () -> stream.read());
+  }
+
+  @Test
+  public void testCloseWithNoAbort() throws Throwable {
+    describe("Close the stream with the readahead outstandingV");
+    FSDataInputStream stream = select(getFileSystem(), csvPath, selectConf,
+        "SELECT * FROM S3OBJECT s");
+    stream.setReadahead(0x1000L);
+    SelectInputStream sis = (SelectInputStream) stream.getWrappedStream();
+    S3AInstrumentation.InputStreamStatistics stats
+        = sis.getS3AStreamStatistics();
+    stream.close();
+    assertEquals("Close count in " + sis, 1, stats.closed);
+    assertEquals("Abort count in " + sis, 0, stats.aborted);
+    assertTrue("No bytes read in close of " + sis, stats.bytesReadInClose > 0);
+  }
+
+  @Test
+  public void testFileContextIntegration() throws Throwable {
+    describe("Test that select works through FileContext");
+    FileContext fc = S3ATestUtils.createTestFileContext(getConfiguration());
+    selectConf.set(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE);
+
+    List<String> selected =
+        verifySelectionCount(ODD_ROWS_COUNT, SELECT_ODD_ENTRIES_INT,
+            parseToLines(
+                select(fc, csvPath, selectConf, SELECT_ODD_ROWS)));
+    // the list includes odd values
+    assertThat(selected, hasItem(ENTRY_0001));
+    // but not the evens
+    assertThat(selected, not(hasItem(ENTRY_0002)));
+  }
+
+  @Test
+  public void testSelectOptionsOnlyOnSelectCalls() throws Throwable {
+    describe("Secondary select options are only valid on select"
+        + " queries");
+    String key = CSV_INPUT_HEADER;
+    intercept(IllegalArgumentException.class, key,
+        () -> getFileSystem().openFile(csvPath)
+            .must(key, CSV_HEADER_OPT_USE).build());
+  }
+
+  @Test
+  public void testSelectMustBeEnabled() throws Throwable {
+    describe("Verify that the FS must have S3 select enabled.");
+    Configuration conf = new Configuration(getFileSystem().getConf());
+    conf.setBoolean(FS_S3A_SELECT_ENABLED, false);
+    try (FileSystem fs2 = FileSystem.newInstance(csvPath.toUri(), conf)) {
+      intercept(UnsupportedOperationException.class,
+          SELECT_UNSUPPORTED,
+          () -> {
+            assertFalse("S3 Select Capability must be disabled on " + fs2,
+                isSelectAvailable(fs2));
+            return fs2.openFile(csvPath)
+              .must(SELECT_SQL, SELECT_ODD_ROWS)
+              .build();
+          });
+    }
+  }
+
+  @Test
+  public void testSelectOptionsRejectedOnNormalOpen() throws Throwable {
+    describe("Verify that a normal open fails on select must() options");
+    intercept(IllegalArgumentException.class,
+        AbstractFSBuilderImpl.UNKNOWN_MANDATORY_KEY,
+        () -> getFileSystem().openFile(csvPath)
+            .must(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE)
+            .build());
+  }
+
+  @Test
+  public void testSelectOddRecordsWithHeader()
+      throws Throwable {
+    describe("work through a record reader");
+    JobConf conf = createJobConf();
+    inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE);
+    expectRecordsRead(ODD_ROWS_COUNT, conf, SELECT_ODD_ENTRIES_DECIMAL);
+  }
+
+  @Test
+  public void testSelectDatestampsConverted()
+      throws Throwable {
+    describe("timestamp conversion in record IIO");
+    JobConf conf = createJobConf();
+    inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE);
+    inputMust(conf, CSV_OUTPUT_QUOTE_FIELDS,
+        CSV_OUTPUT_QUOTE_FIELDS_AS_NEEEDED);
+    String sql = SELECT_TO_DATE;
+    List<String> records = expectRecordsRead(ALL_ROWS_COUNT, conf, sql);
+    LOG.info("Result of {}\n{}", sql, prepareToPrint(records));
+  }
+
+  @Test
+  public void testSelectNoMatch()
+      throws Throwable {
+    describe("when there's no match to a query, 0 records are returned,");
+    JobConf conf = createJobConf();
+    inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE);
+    expectRecordsRead(0, conf,
+        "SELECT * FROM S3OBJECT s WHERE s.odd = " + q("maybe"));
+  }
+
+  @Test
+  public void testSelectOddRecordsIgnoreHeader()
+      throws Throwable {
+    describe("work through a record reader");
+    JobConf conf = createJobConf();
+    inputOpt(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_NONE);
+    inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_IGNORE);
+    expectRecordsRead(EVEN_ROWS_COUNT, conf,
+        SELECT_EVEN_ROWS_NO_HEADER);
+  }
+
+  @Test
+  public void testSelectRecordsUnknownMustOpt()
+      throws Throwable {
+    describe("verify reader key validation is remapped");
+    JobConf conf = createJobConf();
+    inputOpt(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_NONE);
+    inputMust(conf, CSV_INPUT_HEADER + ".something", CSV_HEADER_OPT_IGNORE);
+    intercept(IllegalArgumentException.class,
+        AbstractFSBuilderImpl.UNKNOWN_MANDATORY_KEY,
+        () -> readRecords(conf, SELECT_EVEN_ROWS_NO_HEADER));
+  }
+
+  @Test
+  public void testSelectOddRecordsWithHeaderV1()
+      throws Throwable {
+    describe("work through a V1 record reader");
+    JobConf conf = createJobConf();
+    inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE);
+    // using a double backslash here makes the string "\t" which will then
+    // be parsed in the SelectBinding code as it if had come in on from an XML
+    // entry
+    inputMust(conf, CSV_OUTPUT_FIELD_DELIMITER, "\\t");
+    inputMust(conf, CSV_OUTPUT_QUOTE_CHARACTER, "'");
+    inputMust(conf, CSV_OUTPUT_QUOTE_FIELDS,
+        CSV_OUTPUT_QUOTE_FIELDS_AS_NEEEDED);
+    inputMust(conf, CSV_OUTPUT_RECORD_DELIMITER, "\n");
+    verifySelectionCount(ODD_ROWS_COUNT,
+        SELECT_ODD_ROWS,
+        readRecordsV1(conf, SELECT_ODD_ROWS));
+  }
+
+  /**
+   * Create a job conf for line reader tests.
+   * This patches the job with the passthrough codec for
+   * CSV files.
+   * @return a job configuration
+   */
+  private JobConf createJobConf() {
+    JobConf conf = new JobConf(getConfiguration());
+    enablePassthroughCodec(conf, ".csv");
+    return conf;
+  }
+
+  @Test
+  public void testSelectOddRecordsIgnoreHeaderV1()
+      throws Throwable {
+    describe("work through a V1 record reader");
+    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, SELECT_ERRORS_INCLUDE_SQL, "true");
+    verifySelectionCount(EVEN_ROWS_COUNT,
+        SELECT_EVEN_ROWS_NO_HEADER,
+        readRecordsV1(conf, SELECT_EVEN_ROWS_NO_HEADER));
+  }
+
+  protected List<String> expectRecordsRead(final int expected,
+      final JobConf conf,
+      final String sql) throws Exception {
+    return verifySelectionCount(expected, sql, readRecords(conf, sql));
+  }
+
+  /**
+   * Reads lines through {@link LineRecordReader}, as if it were an MR
+   * job.
+   * @param conf jpb conf
+   * @param sql sql to add to the configuration.
+   * @return the selected lines.
+   * @throws Exception failure
+   */
+  private List<String> readRecords(JobConf conf, String sql) throws Exception {
+    return readRecords(conf,
+        csvPath,
+        sql,
+        createLineRecordReader(),
+        ALL_ROWS_COUNT_WITH_HEADER);
+  }
+
+  /**
+   * Reads lines through a v1 LineRecordReader}.
+   * @param conf jpb conf
+   * @param sql sql to add to the configuration.
+   * @return the selected lines.
+   * @throws Exception failure
+   */
+  private List<String> readRecordsV1(JobConf conf, String sql)
+      throws Exception {
+    inputMust(conf, SELECT_SQL, sql);
+    return super.readRecordsV1(conf,
+        createLineRecordReaderV1(conf, csvPath),
+        new LongWritable(),
+        new Text(),
+        ALL_ROWS_COUNT_WITH_HEADER);
+  }
+
+  /**
+   * Issue a select call, expect the specific number of rows back.
+   * Error text will include the SQL.
+   * @param expected expected row count.
+   * @param conf config for the select call.
+   * @param header header option
+   * @param sql template for a formatted SQL request.
+   * @param args arguments for the formatted request.
+   * @return the lines selected
+   * @throws IOException failure
+   */
+  private List<String> expectSelected(
+      final int expected,
+      final Configuration conf,
+      final String header,
+      final String sql,
+      final Object...args) throws Exception {
+    conf.set(CSV_INPUT_HEADER, header);
+    return verifySelectionCount(expected, sql(sql, args),
+        selectCsvFile(conf, sql, args));
+  }
+
+  /**
+   * Select from the CSV file.
+   * @param conf config for the select call.
+   * @param sql template for a formatted SQL request.
+   * @param args arguments for the formatted request.
+   * @return the lines selected
+   * @throws IOException failure
+   */
+  private List<String> selectCsvFile(
+      final Configuration conf,
+      final String sql,
+      final Object...args)
+      throws Exception {
+
+    return parseToLines(
+        select(getFileSystem(), csvPath, conf, sql, args));
+  }
+
+  @Test
+  public void testCommentsSkipped() throws Throwable {
+    describe("Verify that comments are skipped");
+    selectConf.set(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE);
+
+    List<String> lines = verifySelectionCount(
+        ALL_ROWS_COUNT_WITH_HEADER,
+        "select s.id",
+        parseToLines(
+            select(getFileSystem(), brokenCSV, selectConf,
+                "SELECT * FROM S3OBJECT s")));
+    LOG.info("\n{}", prepareToPrint(lines));
+  }
+
+  @Test
+  public void testEmptyColumnsRegenerated() throws Throwable {
+    describe("if you ask for a column but your row doesn't have it,"
+        + " an empty column is inserted");
+    selectConf.set(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE);
+
+    List<String> lines = verifySelectionCount(
+        ALL_ROWS_COUNT_WITH_HEADER, "select s.oddrange",
+        parseToLines(
+            select(getFileSystem(), brokenCSV, selectConf,
+                "SELECT s.oddrange FROM S3OBJECT s")));
+    LOG.info("\n{}", prepareToPrint(lines));
+    assertEquals("Final oddrange column is not regenerated empty",
+        "\"\"", lines.get(lines.size() - 1));
+  }
+
+  @Test
+  public void testIntCastFailure() throws Throwable {
+    describe("Verify that int casts fail");
+    expectSelectFailure(E_CAST_FAILED, SELECT_ODD_ENTRIES_INT);
+
+  }
+
+  @Test
+  public void testSelectToDateParseFailure() throws Throwable {
+    describe("Verify date parsing failure");
+    expectSelectFailure(E_CAST_FAILED, SELECT_TO_DATE);
+  }
+
+  @Test
+  public void testParseInvalidPathComponent() throws Throwable {
+    describe("Verify bad SQL parseing");
+    expectSelectFailure(E_PARSE_INVALID_PATH_COMPONENT,
+        "SELECT * FROM S3OBJECT WHERE s.'oddf' = true");
+  }
+
+  @Test
+  public void testSelectInvalidTableAlias() throws Throwable {
+    describe("select with unknown column name");
+    expectSelectFailure(E_INVALID_TABLE_ALIAS,
+        "SELECT * FROM S3OBJECT WHERE s.\"oddf\" = 'true'");
+  }
+
+  @Test
+  public void testSelectGeneratedAliases() throws Throwable {
+    describe("select with a ._2 column when headers are enabled");
+    expectSelectFailure(E_INVALID_TABLE_ALIAS,
+        "SELECT * FROM S3OBJECT WHERE s._2 = 'true'");
+  }
+
+  /**
+   * Expect select against the broken CSV file to fail with a specific
+   * AWS exception error code.
+   * If the is no failure, the results are included in the assertion raised.
+   * @param expectedErrorCode error code in getErrorCode()
+   * @param sql SQL to invoke
+   * @return the exception, if it is as expected.
+   * @throws Exception any other failure
+   * @throws AssertionError when an exception is raised, but its error code
+   * is different, or when no exception was raised.
+   */
+  protected AWSServiceIOException expectSelectFailure(
+      String expectedErrorCode,
+      String sql)
+      throws Exception {
+    selectConf.set(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE);
+    return verifyErrorCode(expectedErrorCode,
+        intercept(AWSBadRequestException.class,
+            () ->
... 1071 lines suppressed ...


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