[ 
https://issues.apache.org/jira/browse/HADOOP-16202?focusedWorklogId=536623&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-536623
 ]

ASF GitHub Bot logged work on HADOOP-16202:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 15/Jan/21 20:10
            Start Date: 15/Jan/21 20:10
    Worklog Time Spent: 10m 
      Work Description: ThomasMarquardt commented on a change in pull request 
#2584:
URL: https://github.com/apache/hadoop/pull/2584#discussion_r558505973



##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
##########
@@ -4655,7 +4656,7 @@ public FutureDataInputStreamBuilder openFile(PathHandle 
pathHandle)
       final OpenFileParameters parameters) throws IOException {
     AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(

Review comment:
       This rejectUnknownMandatoryKeys function would not be necessary if the 
mandatory keys were strongly typed fields.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java
##########
@@ -61,6 +61,13 @@
    */
   B opt(@Nonnull String key, float value);
 
+  /**
+   * Set optional long parameter for the Builder.
+   *
+   * @see #opt(String, String)
+   */
+  B opt(@Nonnull String key, long value);
+

Review comment:
       We could use properties (getter and setter) for the standard options. 
We'd have one for buffer size, one for file length, etc.

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,522 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# `FileSystem.openFile()`/`FileContext.openFile()`
+
+This is a method provided by both FileSystem and FileContext for
+advanced file opening options and, where implemented, 
+an asynchrounous/lazy opening of a file.
+
+Creates a builder to open a file, supporting options
+both standard and filesystem specific. The return
+value of the `build()` call is a `Future<FSDataInputStream>`,
+which must be waited on. The file opening may be
+asynchronous, and it may actually be postponed (including
+permission/existence checks) until reads are actually
+performed.
+
+This API call was added to `FileSystem` and `FileContext` in
+Hadoop 3.3.0; it was tuned in Hadoop 3.3.1 as follows.
+
+* Added `opt(key, long)` and `must(key, long)`. 
+* Declared that `withFileStatus(null)` is allowed.
+* Declared that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+* Added standard option keys.
+
+###  <a name="openfile(path)"></a> `FutureDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FutureDataInputStreamBuilder` 
instance,
+the builder parameters are verified and
+`FileSystem.openFileWithOptions(Path, OpenFileParameters)` or
+`AbstractFileSystem.openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+These protected methods returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `FileSystem.openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `FileSystem.open(Path, int)`.
+
+Thus the chain `FileSystem.openFile(path).build().get()` has the same 
preconditions
+and postconditions as `FileSystem.open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of: 
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+The `openFile()` operation MAY check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and
+the actual `build()` and `get()` operations, this file-specific
+preconditions (file exists, file is readable, etc) MUST NOT be checked here.
+
+FileSystem implementations which do not implement `open(Path, int)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FutureDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+### Implementors notes
+
+The base implementation of `openFileWithOptions()` actually executes
+the `open(path)` operation synchronously, yet still returns the result
+or any failures in the `CompletableFuture<>`, so as to ensure that users
+code expecting this.
+
+Any filesystem where the time to open a file may be significant SHOULD
+execute it asynchronously by submitting the operation in some executor/thread
+pool. This is particularly recommended for object stores and other filesystems
+likely to be accessed over long-haul connections.
+
+Arbitrary filesystem-specific options MAY be supported; these MUST
+be prefixed with either the filesystem schema, e.g. `hdfs.`
+or in the `fs.SCHEMA` format as normal configuration settings `fs.hdfs`. The
+latter style allows the same configuration option to be used for both
+filesystem configuration and file-specific configuration.
+
+It SHOULD be possible to always open a file without specifying any options,
+so as to present a consistent model to users. However, an implementation MAY
+opt to require one or more mandatory options to be set.
+
+The returned stream may perform "lazy" evaluation of file access. This is
+relevant for object stores where the probes for existence are expensive, and,
+even with an asynchronous open, may be considered needless.
+ 
+### <a name="openfile(pathhandle)"></a> `FutureDataInputStreamBuilder 
openFile(PathHandle)`
+
+Creates a [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file identified by the given `PathHandle` 
for reading.
+
+When `build()` is invoked on the returned `FutureDataInputStreamBuilder` 
instance,
+the builder parameters are verified and
+`openFileWithOptions(PathHandle, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)` method
+returns a future which invokes `open(Path, int)`.
+
+Thus the chain `openFile(pathhandle).build().get()` has the same preconditions
+and postconditions as `open(Pathhandle, int)`
+
+As with `FutureDataInputStreamBuilder 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
+`FutureDataInputStreamBuilder.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.
+
+
+## <a name="options"></a> Standard `openFile()` options since Hadoop 3.3.1
+
+These are options which FileSystem implementations are expected to
+recognise -and ideally, support by changing the behavior of the input streams
+as a appropriate.
+
+Hadoop 3.3.0 added the `openFile()` API; these standard options were
+defined in Hadoop 3.3.1. Therefore, although they are "well known", unless
+confident that the application will only be executed against releases
+of Hadoop which knows of the options -set the options via `opt()`
+calls rather than `must()`.
+
+When opening a file through the `openFile()` builder API, callers
+MAY use both `.opt(key, value)` and `.must(key, value)` calls to set
+standard and filesystem-specific options. 
+
+If set as an `opt()` parameter, unsupported "standard" options MUST be ignored,
+as MUST unrecognized standard options.
+
+If set as an `must()` parameter, unsupported "standard" options MUST be 
ignored.
+unrecognized standard options MUST be rejected.

Review comment:
       I've read this a few times and it is not clear.  I think you mean to say:
   
   An option set via opt() that is not supported by the filesystem must be 
ignored.  On the other hand, an option set via must() that is not supported by 
the filesystem must result in an exception.

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,522 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# `FileSystem.openFile()`/`FileContext.openFile()`
+
+This is a method provided by both FileSystem and FileContext for
+advanced file opening options and, where implemented, 
+an asynchrounous/lazy opening of a file.
+
+Creates a builder to open a file, supporting options
+both standard and filesystem specific. The return
+value of the `build()` call is a `Future<FSDataInputStream>`,
+which must be waited on. The file opening may be
+asynchronous, and it may actually be postponed (including
+permission/existence checks) until reads are actually
+performed.
+
+This API call was added to `FileSystem` and `FileContext` in
+Hadoop 3.3.0; it was tuned in Hadoop 3.3.1 as follows.
+
+* Added `opt(key, long)` and `must(key, long)`. 
+* Declared that `withFileStatus(null)` is allowed.
+* Declared that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+* Added standard option keys.
+
+###  <a name="openfile(path)"></a> `FutureDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FutureDataInputStreamBuilder` 
instance,
+the builder parameters are verified and
+`FileSystem.openFileWithOptions(Path, OpenFileParameters)` or
+`AbstractFileSystem.openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+These protected methods returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `FileSystem.openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `FileSystem.open(Path, int)`.
+
+Thus the chain `FileSystem.openFile(path).build().get()` has the same 
preconditions
+and postconditions as `FileSystem.open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of: 
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+The `openFile()` operation MAY check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and
+the actual `build()` and `get()` operations, this file-specific
+preconditions (file exists, file is readable, etc) MUST NOT be checked here.
+
+FileSystem implementations which do not implement `open(Path, int)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FutureDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+### Implementors notes
+
+The base implementation of `openFileWithOptions()` actually executes
+the `open(path)` operation synchronously, yet still returns the result
+or any failures in the `CompletableFuture<>`, so as to ensure that users
+code expecting this.
+
+Any filesystem where the time to open a file may be significant SHOULD
+execute it asynchronously by submitting the operation in some executor/thread
+pool. This is particularly recommended for object stores and other filesystems
+likely to be accessed over long-haul connections.
+
+Arbitrary filesystem-specific options MAY be supported; these MUST
+be prefixed with either the filesystem schema, e.g. `hdfs.`
+or in the `fs.SCHEMA` format as normal configuration settings `fs.hdfs`. The
+latter style allows the same configuration option to be used for both
+filesystem configuration and file-specific configuration.
+
+It SHOULD be possible to always open a file without specifying any options,
+so as to present a consistent model to users. However, an implementation MAY
+opt to require one or more mandatory options to be set.
+
+The returned stream may perform "lazy" evaluation of file access. This is
+relevant for object stores where the probes for existence are expensive, and,
+even with an asynchronous open, may be considered needless.
+ 
+### <a name="openfile(pathhandle)"></a> `FutureDataInputStreamBuilder 
openFile(PathHandle)`
+
+Creates a [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file identified by the given `PathHandle` 
for reading.
+
+When `build()` is invoked on the returned `FutureDataInputStreamBuilder` 
instance,
+the builder parameters are verified and
+`openFileWithOptions(PathHandle, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)` method
+returns a future which invokes `open(Path, int)`.
+
+Thus the chain `openFile(pathhandle).build().get()` has the same preconditions
+and postconditions as `open(Pathhandle, int)`
+
+As with `FutureDataInputStreamBuilder 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
+`FutureDataInputStreamBuilder.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.
+
+
+## <a name="options"></a> Standard `openFile()` options since Hadoop 3.3.1
+
+These are options which FileSystem implementations are expected to
+recognise -and ideally, support by changing the behavior of the input streams
+as a appropriate.
+
+Hadoop 3.3.0 added the `openFile()` API; these standard options were
+defined in Hadoop 3.3.1. Therefore, although they are "well known", unless
+confident that the application will only be executed against releases
+of Hadoop which knows of the options -set the options via `opt()`
+calls rather than `must()`.
+
+When opening a file through the `openFile()` builder API, callers
+MAY use both `.opt(key, value)` and `.must(key, value)` calls to set
+standard and filesystem-specific options. 
+
+If set as an `opt()` parameter, unsupported "standard" options MUST be ignored,
+as MUST unrecognized standard options.
+
+If set as an `must()` parameter, unsupported "standard" options MUST be 
ignored.
+unrecognized standard options MUST be rejected.
+
+The standard `openFile()` options are defined in 
`org.apache.hadoop.fs.OpenFileOptions`;
+they all SHALL start with `fs.option.openfile.`.
+
+Note that while all `FileSystem`/`FileContext` instances SHALL support these 
options
+to the extent that `must()` declarations SHALL NOT fail, the implementations
+MAY support them to the extent of interpreting the values.
+This means that it is not a requirement for the stores to actually read the
+the read policy or file length values and use them when opening files.

Review comment:
       Now I'm very confused.  Sounds like options set via must() can also be 
ignored, so why do we have opt() and must()?  
   
   I think we should keep this simple, and only have opt().  Get rid of must(). 
 If a file system does not support the option, it should be ignored.  Nice and 
simple.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java
##########
@@ -38,6 +38,9 @@
    */
   private Set<String> mandatoryKeys;
 
+  /** The optional keys. */
+  private Set<String> optionalKeys;

Review comment:
       Maybe some comments here would help clarify these questions:  What are 
mandatory keys? Why are they mandatory?  What are optional keys and how are 
they different from mandatory keys? I know what these terms mean in English, I 
just didn't catch from the diff why we have these two sets.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
##########
@@ -518,4 +522,112 @@ public String toString() {
     MD5MD5CRC,  // MD5 of block checksums, which are MD5 over chunk CRCs
     COMPOSITE_CRC  // Block/chunk-independent composite CRC
   }
+
+  /**
+   * The standard {@code openFile()} options.
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public static final class OpenFileOptions {
+
+    private OpenFileOptions() {
+    }
+
+    /**
+     * Prefix for all standard filesystem options: {@value}.
+     */
+    public static final String FILESYSTEM_OPTION = "fs.option.";
+
+    /**
+     * Prefix for all openFile options: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE =
+        FILESYSTEM_OPTION + "openfile.";
+
+    /**
+     * OpenFile option for file length: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_LENGTH =
+        FS_OPTION_OPENFILE + "length";
+
+    /**
+     * OpenFile option for split start: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_SPLIT_START =
+        FS_OPTION_OPENFILE + "split.start";
+
+    /**
+     * OpenFile option for split end: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_SPLIT_END =
+        FS_OPTION_OPENFILE + "split.end";
+
+    /**
+     * OpenFile option for buffer size: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_BUFFER_SIZE =
+        FS_OPTION_OPENFILE + "buffer.size";
+
+    /**
+     * OpenFile option for read policies: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY =
+        FS_OPTION_OPENFILE + "read.policy";
+
+    /**
+     * Read policy for adaptive IO: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE =

Review comment:
       Why are so many read policy options needed (adaptive, orc, normal, 
parquet, random, sequential, vectored)?  My initial thought is that sequential, 
random, and default will suffice, where default can be adaptive or a simple 
implementation.  You might want to wait until you have implementations for 
something beyond sequential, random, or default before introducing these other 
options.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
##########
@@ -518,4 +522,112 @@ public String toString() {
     MD5MD5CRC,  // MD5 of block checksums, which are MD5 over chunk CRCs
     COMPOSITE_CRC  // Block/chunk-independent composite CRC
   }
+
+  /**
+   * The standard {@code openFile()} options.
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public static final class OpenFileOptions {
+
+    private OpenFileOptions() {
+    }
+
+    /**
+     * Prefix for all standard filesystem options: {@value}.
+     */
+    public static final String FILESYSTEM_OPTION = "fs.option.";

Review comment:
       I think you could remove this and define the constants below with a 
string literal, or at least make it private.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
##########
@@ -885,7 +885,7 @@ public FutureDataInputStreamBuilder openFile(final Path 
path)
       final OpenFileParameters parameters) throws IOException {
     AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
         parameters.getMandatoryKeys(),
-        Collections.emptySet(),
+        FS_OPTION_OPENFILE_STANDARD_OPTIONS,

Review comment:
       With this change, ChecksumFileSystem.openFileWithOptions has the same 
implementation as the base class, so you can remove this override.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java
##########
@@ -107,16 +109,15 @@ private long dumpFromOffset(PathData item, long offset) 
throws IOException {
     if (offset < 0) {
       offset = Math.max(fileSize + offset, 0);
     }
-    
-    FSDataInputStream in = item.fs.open(item.path);
-    try {
+    // Always do sequential reads.
+    try (FSDataInputStream in = item.openFile(
+        FS_OPTION_OPENFILE_READ_POLICY_NORMAL)) {

Review comment:
       The comment above says do sequential reads, but then the sequential 
option isn't passed here, but seems like it should be.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java
##########
@@ -38,6 +38,9 @@
    */
   private Set<String> mandatoryKeys;

Review comment:
       Should mandatory parameters be strongly typed fields on this class? I 
think it would be better than a Set of strings.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
##########
@@ -390,7 +395,32 @@ public static boolean copy(FileSystem srcFS, Path src,
     return copy(srcFS, fileStatus, dstFS, dst, deleteSource, overwrite, conf);
   }
 
-  /** Copy files between FileSystems. */
+  /**
+   * Copy a file/directory tree within/between filesystems.
+   * <p></p>
+   * returns true if the operation succeeded. When deleteSource is true,
+   * this means "after the copy, delete(source) returned true"
+   * If the destination is a directory, and mkdirs (dest) fails,
+   * the operation will return false rather than raise any exception.
+   * <p></p>
+   * The overwrite flag is about overwriting files; it has no effect about
+   * handing an attempt to copy a file atop a directory (expect an 
IOException),
+   * or a directory over a path which contains a file (mkdir will fail, so
+   * "false").
+   * <p></p>
+   * The operation is recursive, and the deleteSource operation takes place
+   * as each subdirectory is copied. Therefore, if an operation fails partway
+   * through, the source tree may be partially deleted.
+   * @param srcFS source filesystem
+   * @param srcStatus status of source
+   * @param dstFS destination filesystem
+   * @param dst path of source
+   * @param deleteSource delete the source?
+   * @param overwrite overwrite files at destination?
+   * @param conf configuration to use when opening files
+   * @return true if the operation succeeded.

Review comment:
       What is the benefit of returning true on success and throwing on a 
failure, as opposed to returning void and throwing on failure?

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Head.java
##########
@@ -68,11 +70,9 @@ protected void processPath(PathData item) throws IOException 
{
   }
 
   private void dumpToOffset(PathData item) throws IOException {
-    FSDataInputStream in = item.fs.open(item.path);
-    try {
+    try (FSDataInputStream in = item.openFile(
+        FS_OPTION_OPENFILE_READ_POLICY_NORMAL)) {

Review comment:
       Isn't this the same as calling fs.open(Path), as the code was doing 
previously? Shouldn't it be sequential?

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
##########
@@ -518,4 +522,112 @@ public String toString() {
     MD5MD5CRC,  // MD5 of block checksums, which are MD5 over chunk CRCs
     COMPOSITE_CRC  // Block/chunk-independent composite CRC
   }
+
+  /**
+   * The standard {@code openFile()} options.
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public static final class OpenFileOptions {
+
+    private OpenFileOptions() {
+    }
+
+    /**
+     * Prefix for all standard filesystem options: {@value}.
+     */
+    public static final String FILESYSTEM_OPTION = "fs.option.";
+
+    /**
+     * Prefix for all openFile options: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE =
+        FILESYSTEM_OPTION + "openfile.";
+
+    /**
+     * OpenFile option for file length: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_LENGTH =
+        FS_OPTION_OPENFILE + "length";
+
+    /**
+     * OpenFile option for split start: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_SPLIT_START =
+        FS_OPTION_OPENFILE + "split.start";
+
+    /**
+     * OpenFile option for split end: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_SPLIT_END =
+        FS_OPTION_OPENFILE + "split.end";
+
+    /**
+     * OpenFile option for buffer size: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_BUFFER_SIZE =
+        FS_OPTION_OPENFILE + "buffer.size";
+
+    /**
+     * OpenFile option for read policies: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY =
+        FS_OPTION_OPENFILE + "read.policy";
+
+    /**
+     * Read policy for adaptive IO: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY_ADAPTIVE =
+        "adaptive";
+
+    /**
+     * Read policy for ORC files: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY_ORC =
+        "orc";
+
+    /**
+     * Read policy 'normal' -up to implementation: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_READ_POLICY_NORMAL =

Review comment:
       None or default might be a better name for this than normal.

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,522 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# `FileSystem.openFile()`/`FileContext.openFile()`
+
+This is a method provided by both FileSystem and FileContext for
+advanced file opening options and, where implemented, 
+an asynchrounous/lazy opening of a file.
+
+Creates a builder to open a file, supporting options
+both standard and filesystem specific. The return
+value of the `build()` call is a `Future<FSDataInputStream>`,
+which must be waited on. The file opening may be
+asynchronous, and it may actually be postponed (including
+permission/existence checks) until reads are actually
+performed.
+
+This API call was added to `FileSystem` and `FileContext` in
+Hadoop 3.3.0; it was tuned in Hadoop 3.3.1 as follows.
+
+* Added `opt(key, long)` and `must(key, long)`. 
+* Declared that `withFileStatus(null)` is allowed.
+* Declared that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+* Added standard option keys.
+
+###  <a name="openfile(path)"></a> `FutureDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FutureDataInputStreamBuilder` 
instance,
+the builder parameters are verified and
+`FileSystem.openFileWithOptions(Path, OpenFileParameters)` or
+`AbstractFileSystem.openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+These protected methods returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `FileSystem.openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `FileSystem.open(Path, int)`.
+
+Thus the chain `FileSystem.openFile(path).build().get()` has the same 
preconditions
+and postconditions as `FileSystem.open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of: 
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+The `openFile()` operation MAY check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and
+the actual `build()` and `get()` operations, this file-specific
+preconditions (file exists, file is readable, etc) MUST NOT be checked here.
+
+FileSystem implementations which do not implement `open(Path, int)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FutureDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+### Implementors notes
+
+The base implementation of `openFileWithOptions()` actually executes
+the `open(path)` operation synchronously, yet still returns the result
+or any failures in the `CompletableFuture<>`, so as to ensure that users
+code expecting this.
+
+Any filesystem where the time to open a file may be significant SHOULD
+execute it asynchronously by submitting the operation in some executor/thread
+pool. This is particularly recommended for object stores and other filesystems
+likely to be accessed over long-haul connections.
+
+Arbitrary filesystem-specific options MAY be supported; these MUST
+be prefixed with either the filesystem schema, e.g. `hdfs.`
+or in the `fs.SCHEMA` format as normal configuration settings `fs.hdfs`. The
+latter style allows the same configuration option to be used for both
+filesystem configuration and file-specific configuration.
+
+It SHOULD be possible to always open a file without specifying any options,
+so as to present a consistent model to users. However, an implementation MAY
+opt to require one or more mandatory options to be set.
+
+The returned stream may perform "lazy" evaluation of file access. This is
+relevant for object stores where the probes for existence are expensive, and,
+even with an asynchronous open, may be considered needless.
+ 
+### <a name="openfile(pathhandle)"></a> `FutureDataInputStreamBuilder 
openFile(PathHandle)`

Review comment:
       Consider combining the docs for openFile(Path) and openFile(PathHandle) 
to avoid repetition.

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md
##########
@@ -65,27 +65,55 @@ If a filesystem implementation extends the `FileStatus` 
returned in its
 implementation MAY use this information when opening the file.
 
 This is relevant with those stores which return version/etag information,
-including the S3A and ABFS connectors -they MAY use this to guarantee that
-the file they opened is exactly the one returned in the listing.
+-they MAY use this to guarantee that the file they opened
+is exactly the one returned in the listing.
+
+
+The final `status.getPath().getName()` element of the supplied status MUST 
equal
+the name value of the path supplied to the  `openFile(path)` call.
+
+Filesystems MUST NOT validate the rest of the path.
+This is needed to support viewfs and other mount-point wrapper filesystems
+where schemas and paths are different. These often create their own FileStatus 
results
+
+Preconditions
+
+```python
+status == null or status.getPath().getName() == path.getName()
+
+```
+
+Filesystems MUST NOT require the class of `status` to equal
+that of any specific subclass their implementation returns in filestatus/list
+operations. This is to support wrapper filesystems and 
serialization/deserialization
+of the status.
+
 
 ### Set optional or mandatory parameters
 
-    FSDataInputStreamBuilder opt(String key, ...)
-    FSDataInputStreamBuilder must(String key, ...)
+    FutureDataInputStreamBuilder opt(String key, ...)
+    FutureDataInputStreamBuilder must(String key, ...)
 
 Set optional or mandatory parameters to the builder. Using `opt()` or `must()`,
 client can specify FS-specific parameters without inspecting the concrete type
 of `FileSystem`.
 
+Example:
+
 ```java
 out = fs.openFile(path)
-    .opt("fs.s3a.experimental.input.fadvise", "random")
-    .must("fs.s3a.readahead.range", 256 * 1024)
+    .must("fs.option.openfile.read.policy", "random")
+    .opt("fs.http.connection.timeout", 30_000L)
     .withFileStatus(statusFromListing)
     .build()
     .get();
 ```
 
+Here the seek policy of `random` has been specified,

Review comment:
       Now the terminology is "read policy".




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 536623)
    Time Spent: 8h 10m  (was: 8h)

> Stabilize openFile() and adopt internally
> -----------------------------------------
>
>                 Key: HADOOP-16202
>                 URL: https://issues.apache.org/jira/browse/HADOOP-16202
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs, fs/s3, tools/distcp
>    Affects Versions: 3.3.0
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 8h 10m
>  Remaining Estimate: 0h
>
> The {{openFile()}} builder API lets us add new options when reading a file
> Add an option {{"fs.s3a.open.option.length"}} which takes a long and allows 
> the length of the file to be declared. If set, *no check for the existence of 
> the file is issued when opening the file*
> Also: withFileStatus() to take any FileStatus implementation, rather than 
> only S3AFileStatus -and not check that the path matches the path being 
> opened. Needed to support viewFS-style wrapping and mounting.
> and Adopt where appropriate to stop clusters with S3A reads switched to 
> random IO from killing download/localization
> * fs shell copyToLocal
> * distcp
> * IOUtils.copy



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to