Browse Source

HADOOP-16759. Filesystem openFile() builder to take a FileStatus param (#1761). Contributed by Steve Loughran

* Enhanced builder + FS spec
* s3a FS to use this to skip HEAD on open
* and to use version/etag when opening the file

works with S3AFileStatus FS and S3ALocatedFileStatus
Steve Loughran 5 years ago
parent
commit
5e2ce370a3
18 changed files with 544 additions and 141 deletions
  1. 7 9
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
  2. 6 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
  3. 7 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java
  4. 7 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
  5. 21 20
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  6. 5 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
  7. 3 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java
  8. 11 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java
  9. 27 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java
  10. 94 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java
  11. 16 7
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
  12. 41 0
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md
  13. 9 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
  14. 93 27
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  15. 128 13
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java
  16. 38 18
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java
  17. 27 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
  18. 4 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java

+ 7 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java

@@ -31,7 +31,6 @@ 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;
@@ -45,6 +44,7 @@ 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.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -1355,22 +1355,20 @@ public abstract class AbstractFileSystem implements PathCapabilities {
    * 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
+   * @param parameters open file parameters from the builder.
    * @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,
+      final OpenFileParameters parameters) throws IOException {
+    AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
+        parameters.getMandatoryKeys(),
         Collections.emptySet(),
         "for " + path);
     return LambdaUtils.eval(
-        new CompletableFuture<>(), () -> open(path, bufferSize));
+        new CompletableFuture<>(), () ->
+            open(path, parameters.getBufferSize()));
   }
 
   public boolean hasPathCapability(final Path path,

+ 6 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -27,8 +27,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
-import java.util.Locale;
-import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 
 import com.google.common.base.Preconditions;
@@ -37,6 +35,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
 import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl;
+import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.DataChecksum;
@@ -845,14 +844,14 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   @Override
   protected CompletableFuture<FSDataInputStream> openFileWithOptions(
       final Path path,
-      final Set<String> mandatoryKeys,
-      final Configuration options,
-      final int bufferSize) throws IOException {
-    AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(mandatoryKeys,
+      final OpenFileParameters parameters) throws IOException {
+    AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
+        parameters.getMandatoryKeys(),
         Collections.emptySet(),
         "for " + path);
     return LambdaUtils.eval(
-        new CompletableFuture<>(), () -> open(path, bufferSize));
+        new CompletableFuture<>(),
+        () -> open(path, parameters.getBufferSize()));
   }
 
   /**

+ 7 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegateToFileSystem.java

@@ -24,13 +24,13 @@ 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;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
+import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
@@ -266,20 +266,17 @@ public abstract class DelegateToFileSystem extends AbstractFileSystem {
 
   /**
    * Open a file by delegating to
-   * {@link FileSystem#openFileWithOptions(Path, Set, Configuration, int)}.
+   * {@link FileSystem#openFileWithOptions(Path, org.apache.hadoop.fs.impl.OpenFileParameters)}.
    * @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.
+   * @param parameters open file parameters from the builder.
+   *
+   * @return a future which will evaluate to the opened file.ControlAlpha
    * @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);
+      final OpenFileParameters parameters) throws IOException {
+    return fsImpl.openFileWithOptions(path, parameters);
   }
 
   @Override

+ 7 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java

@@ -47,7 +47,7 @@ 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.impl.FsLinkResolution;
-import org.apache.hadoop.fs.impl.PathCapabilitiesSupport;
+import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -2924,16 +2924,18 @@ public class FileContext implements PathCapabilities {
     @Override
     public CompletableFuture<FSDataInputStream> build() throws IOException {
       final Path absF = fixRelativePart(getPath());
+      OpenFileParameters parameters = new OpenFileParameters()
+          .withMandatoryKeys(getMandatoryKeys())
+          .withOptions(getOptions())
+          .withBufferSize(getBufferSize())
+          .withStatus(getStatus());
       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());
+          return fs.openFileWithOptions(p, parameters);
         }
       }.resolve(FileContext.this, absF);
     }

+ 21 - 20
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -58,6 +58,7 @@ 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.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -4485,43 +4486,39 @@ public abstract class FileSystem extends Configured
    * the action of opening the file should begin.
    *
    * The base implementation performs a blocking
-   * call to {@link #open(Path, int)}in this call;
+   * 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
+   * @param parameters open file parameters from the builder.
    * @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,
+      final OpenFileParameters parameters) throws IOException {
+    AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
+        parameters.getMandatoryKeys(),
         Collections.emptySet(),
         "for " + path);
     return LambdaUtils.eval(
-        new CompletableFuture<>(), () -> open(path, bufferSize));
+        new CompletableFuture<>(), () ->
+            open(path, parameters.getBufferSize()));
   }
 
   /**
    * Execute the actual open file operation.
    * The base implementation performs a blocking
-   * call to {@link #open(Path, int)}in this call;
+   * 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
+   * @param parameters open file parameters from the builder.
    * @return a future which will evaluate to the opened file.
    * @throws IOException failure to resolve the link.
    * @throws IllegalArgumentException unknown mandatory key
@@ -4530,14 +4527,13 @@ public abstract class FileSystem extends Configured
    */
   protected CompletableFuture<FSDataInputStream> openFileWithOptions(
       final PathHandle pathHandle,
-      final Set<String> mandatoryKeys,
-      final Configuration options,
-      final int bufferSize) throws IOException {
-    AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(mandatoryKeys,
+      final OpenFileParameters parameters) throws IOException {
+    AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
+        parameters.getMandatoryKeys(),
         Collections.emptySet(), "");
     CompletableFuture<FSDataInputStream> result = new CompletableFuture<>();
     try {
-      result.complete(open(pathHandle, bufferSize));
+      result.complete(open(pathHandle, parameters.getBufferSize()));
     } catch (UnsupportedOperationException tx) {
       // fail fast here
       throw tx;
@@ -4639,12 +4635,17 @@ public abstract class FileSystem extends Configured
     @Override
     public CompletableFuture<FSDataInputStream> build() throws IOException {
       Optional<Path> optionalPath = getOptionalPath();
+      OpenFileParameters parameters = new OpenFileParameters()
+          .withMandatoryKeys(getMandatoryKeys())
+          .withOptions(getOptions())
+          .withBufferSize(getBufferSize())
+          .withStatus(super.getStatus());  // explicit to avoid IDE warnings
       if(optionalPath.isPresent()) {
         return getFS().openFileWithOptions(optionalPath.get(),
-            getMandatoryKeys(), getOptions(), getBufferSize());
+            parameters);
       } else {
         return getFS().openFileWithOptions(getPathHandle(),
-            getMandatoryKeys(), getOptions(), getBufferSize());
+            parameters);
       }
     }
 

+ 5 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -25,12 +25,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.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -714,20 +714,15 @@ public class FilterFileSystem extends FileSystem {
   @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);
+      final OpenFileParameters parameters) throws IOException {
+    return fs.openFileWithOptions(path, parameters);
   }
 
   @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);
+      final OpenFileParameters parameters) throws IOException {
+    return fs.openFileWithOptions(pathHandle, parameters);
   }
 
   @Override

+ 3 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java

@@ -26,13 +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.impl.OpenFileParameters;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -440,10 +439,8 @@ public abstract class FilterFs extends AbstractFileSystem {
   @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);
+      final OpenFileParameters parameters) throws IOException {
+    return myFs.openFileWithOptions(path, parameters);
   }
 
   public boolean hasPathCapability(final Path path,

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java

@@ -47,4 +47,15 @@ public interface FutureDataInputStreamBuilder
   CompletableFuture<FSDataInputStream> build()
       throws IllegalArgumentException, UnsupportedOperationException,
       IOException;
+
+  /**
+   * A FileStatus may be provided to the open request.
+   * It is up to the implementation whether to use this or not.
+   * @param status status.
+   * @return the builder.
+   */
+  default FutureDataInputStreamBuilder withFileStatus(FileStatus status) {
+    return this;
+  }
+
 }

+ 27 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java

@@ -26,12 +26,13 @@ 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.FileStatus;
 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 java.util.Objects.requireNonNull;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
 
@@ -60,6 +61,12 @@ public abstract class FutureDataInputStreamBuilderImpl
 
   private int bufferSize;
 
+  /**
+   * File status passed in through a {@link #withFileStatus(FileStatus)}
+   * call; null otherwise.
+   */
+  private FileStatus status;
+
   /**
    * Construct from a {@link FileContext}.
    *
@@ -69,8 +76,8 @@ public abstract class FutureDataInputStreamBuilderImpl
    */
   protected FutureDataInputStreamBuilderImpl(@Nonnull FileContext fc,
       @Nonnull Path path) throws IOException {
-    super(checkNotNull(path));
-    checkNotNull(fc);
+    super(requireNonNull(path, "path"));
+    requireNonNull(fc, "file context");
     this.fileSystem = null;
     bufferSize = IO_FILE_BUFFER_SIZE_DEFAULT;
   }
@@ -82,8 +89,8 @@ public abstract class FutureDataInputStreamBuilderImpl
    */
   protected FutureDataInputStreamBuilderImpl(@Nonnull FileSystem fileSystem,
       @Nonnull Path path) {
-    super(checkNotNull(path));
-    this.fileSystem = checkNotNull(fileSystem);
+    super(requireNonNull(path, "path"));
+    this.fileSystem = requireNonNull(fileSystem, "fileSystem");
     initFromFS();
   }
 
@@ -108,7 +115,7 @@ public abstract class FutureDataInputStreamBuilderImpl
   }
 
   protected FileSystem getFS() {
-    checkNotNull(fileSystem);
+    requireNonNull(fileSystem, "fileSystem");
     return fileSystem;
   }
 
@@ -138,4 +145,18 @@ public abstract class FutureDataInputStreamBuilderImpl
   public FutureDataInputStreamBuilder getThisBuilder() {
     return this;
   }
+
+  @Override
+  public FutureDataInputStreamBuilder withFileStatus(FileStatus st) {
+    this.status = requireNonNull(st, "status");
+    return this;
+  }
+
+  /**
+   * Get any status set in {@link #withFileStatus(FileStatus)}.
+   * @return a status value or null.
+   */
+  protected FileStatus getStatus() {
+    return status;
+  }
 }

+ 94 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java

@@ -0,0 +1,94 @@
+/*
+ * 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.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * All the parameters from the openFile builder for the
+ * {@code openFileWithOptions} commands.
+ *
+ * If/when new attributes added to the builder, this class will be extended.
+ */
+public class OpenFileParameters {
+
+  /**
+   * Set of options declared as mandatory.
+   */
+  private Set<String> mandatoryKeys;
+
+  /**
+   * Options set during the build sequence.
+   */
+  private Configuration options;
+
+  /**
+   * Buffer size.
+   */
+  private int bufferSize;
+
+  /**
+   * Optional file status.
+   */
+  private FileStatus status;
+
+  public OpenFileParameters() {
+  }
+
+  public OpenFileParameters withMandatoryKeys(final Set<String> keys) {
+    this.mandatoryKeys = requireNonNull(keys);
+    return this;
+  }
+
+  public OpenFileParameters withOptions(final Configuration opts) {
+    this.options = requireNonNull(opts);
+    return this;
+  }
+
+  public OpenFileParameters withBufferSize(final int size) {
+    this.bufferSize = size;
+    return this;
+  }
+
+  public OpenFileParameters withStatus(final FileStatus st) {
+    this.status = st;
+    return this;
+  }
+
+  public Set<String> getMandatoryKeys() {
+    return mandatoryKeys;
+  }
+
+  public Configuration getOptions() {
+    return options;
+  }
+
+  public int getBufferSize() {
+    return bufferSize;
+  }
+
+  public FileStatus getStatus() {
+    return status;
+  }
+}

+ 16 - 7
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md

@@ -718,24 +718,29 @@ exists in the metadata, but no copies of any its blocks can be located;
 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.
+`openFileWithOptions(Path, OpenFileParameters)` invoked.
 
 This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
 which, when its `get()` method is called, either returns an input
 stream of the contents of opened file, or raises an exception.
 
-The base implementation of the `openFileWithOptions(PathHandle, Set<String>, Configuration, int)`
+The base implementation of the `openFileWithOptions(PathHandle, OpenFileParameters)`
 ultimately invokes `open(Path, int)`.
 
 Thus the chain `openFile(path).build().get()` has the same preconditions
 and postconditions as `open(Path p, int bufferSize)`
 
+However, there is one difference which implementations are free to
+take advantage of: 
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
 
-The `openFile()` operation may check the state of the filesystem during this
-call, but as the state of the filesystem may change betwen this call and
+The `openFile()` operation may check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and
 the actual `build()` and `get()` operations, this file-specific
 preconditions (file exists, file is readable, etc) MUST NOT be checked here.
 
@@ -766,6 +771,10 @@ It SHOULD be possible to always open a file without specifying any options,
 so as to present a consistent model to users. However, an implementation MAY
 opt to require one or more mandatory options to be set.
 
+The returned stream may perform "lazy" evaluation of file access. This is
+relevant for object stores where the probes for existence are expensive, and,
+even with an asynchronous open, may be considered needless.
+ 
 ### `FSDataInputStreamBuilder openFile(PathHandle)`
 
 Creates a `FSDataInputStreamBuilder` to build an operation to open a file.
@@ -774,13 +783,13 @@ to construct a operation to open the file identified by the given `PathHandle` f
 
 When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
 the builder parameters are verified and
-`openFileWithOptions(PathHandle, Set<String>, Configuration, int)` invoked.
+`openFileWithOptions(PathHandle, OpenFileParameters)` invoked.
 
 This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
 which, when its `get()` method is called, either returns an input
 stream of the contents of opened file, or raises an exception.
 
-The base implementation of the `openFileWithOptions(Path,PathHandle, Set<String>, Configuration, int)` method
+The base implementation of the `openFileWithOptions(PathHandle, OpenFileParameters)` method
 returns a future which invokes `open(Path, int)`.
 
 Thus the chain `openFile(pathhandle).build().get()` has the same preconditions

+ 41 - 0
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md

@@ -43,6 +43,31 @@ path validation.
 
 Set the size of the buffer to be used.
 
+### <a name="Builder.withFileStatus"></a> `FSDataInputStreamBuilder withFileStatus(FileStatus status)`
+
+A `FileStatus` instance which refers to the file being opened.
+
+This MAY be used by implementations to short-circuit checks for the file,
+So potentially saving on remote calls especially to object stores.
+
+Requirements:
+
+* `status != null`
+* `status.getPath()` == the resolved path of the file being opened.
+
+The path validation MUST take place if the store uses the `FileStatus` when
+it opens files, and MAY be performed otherwise. The validation
+SHOULD be postponed until the `build()` operation.
+
+This operation should be considered a hint to the filesystem.
+
+If a filesystem implementation extends the `FileStatus` returned in its
+implementation MAY use this information when opening the file.
+
+This is relevant with those stores which return version/etag information,
+including the S3A and ABFS connectors -they MAY use this to guarantee that
+the file they opened is exactly the one returned in the listing.
+
 ### Set optional or mandatory parameters
 
     FSDataInputStreamBuilder opt(String key, ...)
@@ -56,6 +81,7 @@ of `FileSystem`.
 out = fs.openFile(path)
     .opt("fs.s3a.experimental.input.fadvise", "random")
     .must("fs.s3a.readahead.range", 256 * 1024)
+    .withFileStatus(statusFromListing)
     .build()
     .get();
 ```
@@ -76,6 +102,21 @@ builder methods (i.e., `bufferSize()`) and `opt()`/`must()` is as follows:
 
 > The last option specified defines the value and its optional/mandatory state.
 
+If the `FileStatus` option passed in `withFileStatus()` is used, implementations
+MUST accept all subclasses of `FileStatus`, including `LocatedFileStatus`,
+rather than just any FS-specific subclass implemented by the implementation
+(e.g `S3AFileStatus`). They MAY simply ignore those which are not the 
+custom subclasses.
+
+This is critical to ensure safe use of the feature: directory listing/
+status serialization/deserialization can result result in the `withFileStatus()`
+argumennt not being the custom subclass returned by the Filesystem instance's
+own `getFileStatus()`, `listFiles()`, `listLocatedStatus()` calls, etc.
+
+In such a situation the implementations must:
+
+1. Validate the path (always).
+1. Use the status/convert to the custom type, *or* simply discard it.
 
 ## Builder interface
 

+ 9 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java

@@ -281,6 +281,7 @@ public abstract class AbstractContractOpenTest
     createFile(fs, path, true,
         dataset(len, 0x40, 0x80));
     CompletableFuture<Long> readAllBytes = fs.openFile(path)
+        .withFileStatus(fs.getFileStatus(path))
         .build()
         .thenApply(ContractTestUtils::readStream);
     assertEquals("Wrong number of bytes read value",
@@ -302,4 +303,12 @@ public abstract class AbstractContractOpenTest
         accepted.get());
   }
 
+  @Test
+  public void testOpenFileNullStatus() throws Throwable {
+    describe("use openFile() with a null status");
+    Path path = path("testOpenFileNullStatus");
+    intercept(NullPointerException.class,
+        () -> getFileSystem().openFile(path).withFileStatus(null));
+  }
+
 }

+ 93 - 27
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -95,6 +95,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Globber;
+import org.apache.hadoop.fs.impl.OpenFileParameters;
 import org.apache.hadoop.fs.s3a.auth.SignerManager;
 import org.apache.hadoop.fs.s3a.auth.delegation.DelegationOperations;
 import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider;
@@ -978,27 +979,30 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @Retries.RetryTranslated
   public FSDataInputStream open(Path f, int bufferSize)
       throws IOException {
-    return open(f, Optional.empty());
+    return open(f, Optional.empty(), Optional.empty());
   }
 
   /**
    * Opens an FSDataInputStream at the indicated Path.
-   * @param path the file to open
+   * if status contains an S3AFileStatus reference, it is used
+   * and so a HEAD request to the store is avoided.
+   *
+   * @param file the file to open
    * @param options configuration options if opened with the builder API.
+   * @param providedStatus optional file status.
    * @throws IOException IO failure.
    */
   @Retries.RetryTranslated
   private FSDataInputStream open(
-      final Path path,
-      final Optional<Configuration> options)
+      final Path file,
+      final Optional<Configuration> options,
+      final Optional<S3AFileStatus> providedStatus)
       throws IOException {
 
     entryPoint(INVOCATION_OPEN);
-    final S3AFileStatus fileStatus = (S3AFileStatus) getFileStatus(path);
-    if (fileStatus.isDirectory()) {
-      throw new FileNotFoundException("Can't open " + path
-          + " because it is a directory");
-    }
+    final Path path = qualify(file);
+    S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path,
+        providedStatus);
 
     S3AReadOpContext readContext;
     if (options.isPresent()) {
@@ -4303,22 +4307,21 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    * @param source path to source data
    * @param expression select expression
    * @param options request configuration from the builder.
+   * @param providedStatus any passed in status
    * @return the stream of the results
    * @throws IOException IO failure
    */
   @Retries.RetryTranslated
   private FSDataInputStream select(final Path source,
       final String expression,
-      final Configuration options)
+      final Configuration options,
+      final Optional<S3AFileStatus> providedStatus)
       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 S3AFileStatus fileStatus = (S3AFileStatus) getFileStatus(path);
+    final S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path,
+        providedStatus);
 
     // readahead range can be dynamically set
     long ra = options.getLong(READAHEAD_RANGE, readAhead);
@@ -4326,10 +4329,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     S3AReadOpContext readContext = createReadContext(fileStatus, inputPolicy,
         changeDetectionPolicy, ra);
 
-    if (!fileStatus.isDirectory()) {
+    if (changeDetectionPolicy.getSource() != ChangeDetectionPolicy.Source.None
+        && fileStatus.getETag() != null) {
+      // if there is change detection, and the status includes at least an
+      // etag,
       // check that the object metadata lines up with what is expected
       // based on the object attributes (which may contain an eTag or
-      // versionId) from S3Guard
+      // versionId).
+      // This is because the select API doesn't offer this.
+      // (note: this is trouble for version checking as cannot force the old
+      // version in the final read; nor can we check the etag match)
       ChangeTracker changeTracker =
           new ChangeTracker(uri.toString(),
               changeDetectionPolicy,
@@ -4364,12 +4373,42 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     }
   }
 
+  /**
+   * Extract the status from the optional parameter, querying
+   * S3Guard/s3 if it is absent.
+   * @param path path of the status
+   * @param optStatus optional status
+   * @return a file status
+   * @throws FileNotFoundException if there is no normal file at that path
+   * @throws IOException IO failure
+   */
+  private S3AFileStatus extractOrFetchSimpleFileStatus(
+      final Path path, final Optional<S3AFileStatus> optStatus)
+      throws IOException {
+    S3AFileStatus fileStatus;
+    if (optStatus.isPresent()) {
+      fileStatus = optStatus.get();
+    } else {
+      // this looks at S3guard and gets any type of status back,
+      // if it falls back to S3 it does a HEAD only.
+      // therefore: if there is no S3Guard and there is a dir, this
+      // will raise a FileNotFoundException
+      fileStatus = innerGetFileStatus(path, false,
+          StatusProbeEnum.HEAD_ONLY);
+    }
+    // we check here for the passed in status or the S3Guard value
+    // for being a directory
+    if (fileStatus.isDirectory()) {
+      throw new FileNotFoundException(path.toString() + " is a directory");
+    }
+    return fileStatus;
+  }
+
   /**
    * 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.
+   * @param rawPath path to the file
+   * @param parameters open file parameters from the builder.
    * @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
@@ -4379,10 +4418,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   @Override
   @Retries.RetryTranslated
   public CompletableFuture<FSDataInputStream> openFileWithOptions(
-      final Path path,
-      final Set<String> mandatoryKeys,
-      final Configuration options,
-      final int bufferSize) throws IOException {
+      final Path rawPath,
+      final OpenFileParameters parameters) throws IOException {
+    final Path path = qualify(rawPath);
+    Configuration options = parameters.getOptions();
+    Set<String> mandatoryKeys = parameters.getMandatoryKeys();
     String sql = options.get(SelectConstants.SELECT_SQL, null);
     boolean isSelect = sql != null;
     // choice of keys depends on open type
@@ -4397,20 +4437,46 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           InternalConstants.STANDARD_OPENFILE_KEYS,
           "for " + path + " in non-select file I/O");
     }
+    FileStatus providedStatus = parameters.getStatus();
+    S3AFileStatus fileStatus;
+    if (providedStatus != null) {
+      Preconditions.checkArgument(path.equals(providedStatus.getPath()),
+          "FileStatus parameter is not for the path %s: %s",
+          path, providedStatus);
+      if (providedStatus instanceof S3AFileStatus) {
+        // can use this status to skip our own probes,
+        // including etag and version.
+        LOG.debug("File was opened with a supplied S3AFileStatus;"
+            + " skipping getFileStatus call in open() operation: {}",
+            providedStatus);
+        fileStatus = (S3AFileStatus) providedStatus;
+      } else if (providedStatus instanceof S3ALocatedFileStatus) {
+        LOG.debug("File was opened with a supplied S3ALocatedFileStatus;"
+            + " skipping getFileStatus call in open() operation: {}",
+            providedStatus);
+        fileStatus = ((S3ALocatedFileStatus) providedStatus).toS3AFileStatus();
+      } else {
+        LOG.debug("Ignoring file status {}", providedStatus);
+        fileStatus = null;
+      }
+    } else {
+      fileStatus = null;
+    }
+    Optional<S3AFileStatus> ost = Optional.ofNullable(fileStatus);
     CompletableFuture<FSDataInputStream> result = new CompletableFuture<>();
     if (!isSelect) {
       // normal path.
       unboundedThreadPool.submit(() ->
           LambdaUtils.eval(result,
-              () -> open(path, Optional.of(options))));
+              () -> open(path, Optional.of(options), ost)));
     } else {
       // it is a select statement.
-      // fail fast if the method is not present
+      // fail fast if the operation is not available
       requireSelectSupport(path);
       // submit the query
       unboundedThreadPool.submit(() ->
           LambdaUtils.eval(result,
-              () -> select(path, sql, options)));
+              () -> select(path, sql, options, ost)));
     }
     return result;
   }

+ 128 - 13
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.fs.s3a;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Optional;
@@ -432,6 +432,106 @@ public class ITestS3ARemoteFileChanged extends AbstractS3ATestBase {
     }
   }
 
+  /**
+   * Verifies that when the openFile builder is passed in a status,
+   * then that is used to eliminate the getFileStatus call in open();
+   * thus the version and etag passed down are still used.
+   */
+  @Test
+  public void testOpenFileWithStatus() throws Throwable {
+    final Path testpath = path("testOpenFileWithStatus.dat");
+    final byte[] dataset = TEST_DATA_BYTES;
+    S3AFileStatus originalStatus =
+        writeFile(testpath, dataset, dataset.length, true);
+
+    // forge a file status with a different etag
+    // no attempt is made to change the versionID as it will
+    // get rejected by S3 as an invalid version
+    S3AFileStatus forgedStatus =
+        S3AFileStatus.fromFileStatus(originalStatus, Tristate.FALSE,
+            originalStatus.getETag() + "-fake",
+            originalStatus.getVersionId() + "");
+    fs.getMetadataStore().put(
+        new PathMetadata(forgedStatus, Tristate.FALSE, false));
+
+    // verify the bad etag gets picked up.
+    LOG.info("Opening stream with s3guard's (invalid) status.");
+    try (FSDataInputStream instream = fs.openFile(testpath)
+        .build()
+        .get()) {
+      try {
+        instream.read();
+        // No exception only if we don't enforce change detection as exception
+        assertTrue(
+            "Read did not raise an exception even though the change detection "
+                + "mode was " + changeDetectionMode
+                + " and the inserted file status was invalid",
+            changeDetectionMode.equals(CHANGE_DETECT_MODE_NONE)
+                || changeDetectionMode.equals(CHANGE_DETECT_MODE_WARN)
+                || changeDetectionSource.equals(CHANGE_DETECT_SOURCE_VERSION_ID));
+      } catch (RemoteFileChangedException ignored) {
+        // Ignored.
+      }
+    }
+
+    // By passing in the status open() doesn't need to check s3guard
+    // And hence the existing file is opened
+    LOG.info("Opening stream with the original status.");
+    try (FSDataInputStream instream = fs.openFile(testpath)
+        .withFileStatus(originalStatus)
+        .build()
+        .get()) {
+      instream.read();
+    }
+
+    // and this holds for S3A Located Status
+    LOG.info("Opening stream with S3ALocatedFileStatus.");
+    try (FSDataInputStream instream = fs.openFile(testpath)
+        .withFileStatus(new S3ALocatedFileStatus(originalStatus, null))
+        .build()
+        .get()) {
+      instream.read();
+    }
+
+    // if you pass in a status of a dir, it will be rejected
+    S3AFileStatus s2 = new S3AFileStatus(true, testpath, "alice");
+    assertTrue("not a directory " + s2, s2.isDirectory());
+    LOG.info("Open with directory status");
+    interceptFuture(FileNotFoundException.class, "",
+        fs.openFile(testpath)
+            .withFileStatus(s2)
+            .build());
+
+    // now, we delete the file from the store and s3guard
+    // when we pass in the status, there's no HEAD request, so it's only
+    // in the read call where the 404 surfaces.
+    // and there, when versionID is passed to the GET, the data is returned
+    LOG.info("Testing opening a deleted file");
+    fs.delete(testpath, false);
+    try (FSDataInputStream instream = fs.openFile(testpath)
+        .withFileStatus(originalStatus)
+        .build()
+        .get()) {
+      if (changeDetectionSource.equals(CHANGE_DETECT_SOURCE_VERSION_ID)
+          && changeDetectionMode.equals(CHANGE_DETECT_MODE_SERVER)) {
+          // the deleted file is still there if you know the version ID
+          // and the check is server-side
+          instream.read();
+      } else {
+        // all other cases, the read will return 404.
+        intercept(FileNotFoundException.class,
+            () -> instream.read());
+      }
+
+    }
+
+    // whereas without that status, you fail in the get() when a HEAD is
+    // issued
+    interceptFuture(FileNotFoundException.class, "",
+        fs.openFile(testpath).build());
+
+  }
+
   /**
    * Ensures a file can be read when there is no version metadata
    * (ETag, versionId).
@@ -524,9 +624,11 @@ public class ITestS3ARemoteFileChanged extends AbstractS3ATestBase {
         writeFileWithNoVersionMetadata("selectnoversion.dat");
 
     try (FSDataInputStream instream = fs.openFile(testpath)
-        .must(SELECT_SQL, "SELECT * FROM S3OBJECT").build().get()) {
+        .must(SELECT_SQL, "SELECT * FROM S3OBJECT")
+        .build()
+        .get()) {
       assertEquals(QUOTED_TEST_DATA,
-          IOUtils.toString(instream, Charset.forName("UTF-8")).trim());
+          IOUtils.toString(instream, StandardCharsets.UTF_8).trim());
     }
   }
 
@@ -902,15 +1004,12 @@ public class ITestS3ARemoteFileChanged extends AbstractS3ATestBase {
   private Path writeOutOfSyncFileVersion(String filename) throws IOException {
     final Path testpath = path(filename);
     final byte[] dataset = TEST_DATA_BYTES;
-    writeDataset(fs, testpath, dataset, dataset.length,
-        1024, false);
-    S3AFileStatus originalStatus = (S3AFileStatus) fs.getFileStatus(testpath);
+    S3AFileStatus originalStatus =
+        writeFile(testpath, dataset, dataset.length, false);
 
     // overwrite with half the content
-    writeDataset(fs, testpath, dataset, dataset.length / 2,
-        1024, true);
-
-    S3AFileStatus newStatus = (S3AFileStatus) fs.getFileStatus(testpath);
+    S3AFileStatus newStatus = writeFile(testpath, dataset, dataset.length / 2,
+        true);
 
     // put back the original etag, versionId
     S3AFileStatus forgedStatus =
@@ -922,6 +1021,23 @@ public class ITestS3ARemoteFileChanged extends AbstractS3ATestBase {
     return testpath;
   }
 
+  /**
+   * Write data to a file; return the status from the filesystem.
+   * @param path file path
+   * @param dataset dataset to write from
+   * @param length number of bytes from the dataset to write.
+   * @param overwrite overwrite flag
+   * @return the retrieved file status.
+   */
+  private S3AFileStatus writeFile(final Path path,
+      final byte[] dataset,
+      final int length,
+      final boolean overwrite) throws IOException {
+    writeDataset(fs, path, dataset, length,
+        1024, overwrite);
+    return (S3AFileStatus) fs.getFileStatus(path);
+  }
+
   /**
    * Writes {@link #TEST_DATA} to a file where the file will be inconsistent
    * in S3 for a set of operations.
@@ -1208,9 +1324,8 @@ public class ITestS3ARemoteFileChanged extends AbstractS3ATestBase {
   private Path writeFileWithNoVersionMetadata(String filename)
       throws IOException {
     final Path testpath = path(filename);
-    writeDataset(fs, testpath, TEST_DATA_BYTES, TEST_DATA_BYTES.length,
-        1024, false);
-    S3AFileStatus originalStatus = (S3AFileStatus) fs.getFileStatus(testpath);
+    S3AFileStatus originalStatus = writeFile(testpath, TEST_DATA_BYTES,
+        TEST_DATA_BYTES.length, false);
 
     // remove ETag and versionId
     S3AFileStatus newStatus = S3AFileStatus.fromFileStatus(originalStatus,

+ 38 - 18
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java

@@ -28,13 +28,13 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.Collectors;
 
 import org.assertj.core.api.Assertions;
-import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
 import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -42,7 +42,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
 import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source;
 import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
 import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
@@ -60,15 +59,17 @@ import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE;
 import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_METADATA_TTL;
 import static org.apache.hadoop.fs.s3a.Constants.RETRY_INTERVAL;
 import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT;
+import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_CONSISTENCY_RETRY_INTERVAL;
+import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_CONSISTENCY_RETRY_LIMIT;
 import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.PROBE_INTERVAL_MILLIS;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.STABILIZATION_TIME;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.TIMESTAMP_SLEEP;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.awaitDeletedFileDisappearance;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.awaitFileStatus;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.checkListingContainsPath;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.checkListingDoesNotContainPath;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.read;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.readWithStatus;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.test.LambdaTestUtils.eventually;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -163,9 +164,14 @@ public class ITestS3GuardOutOfBandOperations extends AbstractS3ATestBase {
     // speeding up the tests
     removeBaseAndBucketOverrides(conf,
         RETRY_LIMIT,
-        RETRY_INTERVAL);
+        RETRY_INTERVAL,
+        S3GUARD_CONSISTENCY_RETRY_INTERVAL,
+        S3GUARD_CONSISTENCY_RETRY_LIMIT);
     conf.setInt(RETRY_LIMIT, 3);
-    conf.set(RETRY_INTERVAL, "10ms");
+    conf.setInt(S3GUARD_CONSISTENCY_RETRY_LIMIT, 3);
+    final String delay = "10ms";
+    conf.set(RETRY_INTERVAL, delay);
+    conf.set(S3GUARD_CONSISTENCY_RETRY_INTERVAL, delay);
     return conf;
   }
 
@@ -284,11 +290,6 @@ public class ITestS3GuardOutOfBandOperations extends AbstractS3ATestBase {
 
   @Test
   public void testOutOfBandDeletes() throws Exception {
-    ChangeDetectionPolicy changeDetectionPolicy =
-        ((S3AFileSystem) getFileSystem()).getChangeDetectionPolicy();
-    Assume.assumeFalse("FNF not expected when using a bucket with"
-            + " object versioning",
-        changeDetectionPolicy.getSource() == Source.VersionId);
 
     Path testFileName = path("OutOfBandDelete-" + UUID.randomUUID());
     outOfBandDeletes(testFileName, authoritative);
@@ -658,8 +659,22 @@ public class ITestS3GuardOutOfBandOperations extends AbstractS3ATestBase {
       FileStatus status = guardedFs.getFileStatus(testFilePath);
       LOG.info("Authoritative: {} status path: {}",
           allowAuthoritative, status.getPath());
-      expectExceptionWhenReading(testFilePath, text);
-      expectExceptionWhenReadingOpenFileAPI(testFilePath, text);
+      final boolean versionedChangeDetection =
+          getFileSystem().getChangeDetectionPolicy().getSource()
+              == Source.VersionId;
+      if (!versionedChangeDetection) {
+        expectExceptionWhenReading(testFilePath, text);
+        expectExceptionWhenReadingOpenFileAPI(testFilePath, text, null);
+        expectExceptionWhenReadingOpenFileAPI(testFilePath, text, status);
+      } else {
+        // FNFE not expected when using a bucket with object versioning
+        final String read1 = read(guardedFs, testFilePath);
+        assertEquals("File read from the auth FS", text, read1);
+        // and when the status is passed in, even the raw FS will ask for it
+        // via the versionId in the status
+        final String read2 = readWithStatus(rawFS, status);
+        assertEquals("File read from the raw FS", text, read2);
+      }
     } finally {
       guardedFs.delete(testFilePath, true);
     }
@@ -957,7 +972,8 @@ public class ITestS3GuardOutOfBandOperations extends AbstractS3ATestBase {
       FileStatus status = guardedFs.getFileStatus(testFilePath);
       LOG.info("authoritative: {} status: {}", allowAuthoritative, status);
       expectExceptionWhenReading(testFilePath, text);
-      expectExceptionWhenReadingOpenFileAPI(testFilePath, text);
+      expectExceptionWhenReadingOpenFileAPI(testFilePath, text, null);
+      expectExceptionWhenReadingOpenFileAPI(testFilePath, text, status);
     } finally {
       guardedFs.delete(testDirPath, true);
     }
@@ -983,14 +999,18 @@ public class ITestS3GuardOutOfBandOperations extends AbstractS3ATestBase {
    * We expect the read to fail with an FNFE: open will be happy.
    * @param testFilePath path of the test file
    * @param text the context in the file.
+   * @param status optional status for the withFileStatus operation.
    * @throws Exception failure other than the FNFE
    */
   private void expectExceptionWhenReadingOpenFileAPI(
-      Path testFilePath, String text)
+      Path testFilePath, String text, FileStatus status)
       throws Exception {
-    try (
-        FSDataInputStream in = guardedFs.openFile(testFilePath).build().get()
-    ) {
+    final FutureDataInputStreamBuilder builder
+        = guardedFs.openFile(testFilePath);
+    if (status != null) {
+      builder.withFileStatus(status);
+    }
+    try (FSDataInputStream in = builder.build().get()) {
       intercept(FileNotFoundException.class, () -> {
         byte[] bytes = new byte[text.length()];
         return in.read(bytes, 0, bytes.length);

+ 27 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

@@ -66,12 +66,14 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
 import java.util.stream.Collectors;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CREDENTIAL_PROVIDER_PATH;
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
+import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture;
 import static org.apache.hadoop.fs.s3a.FailureInjectionPolicy.*;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
@@ -1184,6 +1186,31 @@ public final class S3ATestUtils {
     }
   }
 
+  /**
+   * Read in a file and convert to an ascii string, using the openFile
+   * builder API and the file status.
+   * If the status is an S3A FileStatus, any etag or versionId used
+   * will be picked up.
+   * @param fs filesystem
+   * @param status file status, including path
+   * @return the bytes read and converted to a string
+   * @throws IOException IO problems
+   */
+  public static String readWithStatus(
+      final FileSystem fs,
+      final FileStatus status) throws IOException {
+    final CompletableFuture<FSDataInputStream> future =
+        fs.openFile(status.getPath())
+            .withFileStatus(status)
+            .build();
+
+    try (FSDataInputStream in = awaitFuture(future)) {
+      byte[] buf = new byte[(int) status.getLen()];
+      in.readFully(0, buf);
+      return new String(buf);
+    }
+  }
+
   /**
    * List a directory/directory tree.
    * @param fileSystem FS

+ 4 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java

@@ -256,6 +256,7 @@ public class ITestS3Select extends AbstractS3SelectTest {
     ContractTestUtils.touch(fs, path);
     parseToLines(fs.openFile(path)
             .must(SELECT_SQL, SELECT_EVERYTHING)
+            .withFileStatus(fs.getFileStatus(path))
             .build()
             .get(),
         0);
@@ -548,14 +549,14 @@ public class ITestS3Select extends AbstractS3SelectTest {
     FutureDataInputStreamBuilder builder =
         getFileSystem().openFile(dir)
             .must(SELECT_SQL, SELECT_ODD_ENTRIES);
-    interceptFuture(PathIOException.class,
+    interceptFuture(FileNotFoundException.class,
         "", builder.build());
 
     // try the parent
     builder = getFileSystem().openFile(dir.getParent())
             .must(SELECT_SQL,
                 SELECT_ODD_ENTRIES);
-    interceptFuture(PathIOException.class,
+    interceptFuture(FileNotFoundException.class,
         "", builder.build());
   }
 
@@ -565,7 +566,7 @@ public class ITestS3Select extends AbstractS3SelectTest {
     FutureDataInputStreamBuilder builder =
         getFileSystem().openFile(path("/"))
             .must(SELECT_SQL, SELECT_ODD_ENTRIES);
-    interceptFuture(PathIOException.class,
+    interceptFuture(FileNotFoundException.class,
         "", builder.build());
   }