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

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

                Author: ASF GitHub Bot
            Created on: 09/Dec/20 03:30
            Start Date: 09/Dec/20 03:30
    Worklog Time Spent: 10m 
      Work Description: ThomasMarquardt commented on a change in pull request 
#2168:
URL: https://github.com/apache/hadoop/pull/2168#discussion_r538788336



##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create 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.
+
+* Support `opt(key, long)` and `must(key, long)`. 
+* Declare that `withFileStatus(null)` is allowed.
+* Declare that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+
+
+###  <a name="openfile(path)"></a> `FSDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(Path, OpenFileParameters)` invoked.

Review comment:
       Might be helpful to point out that openFileWithOptions is a method on 
FileSystem.

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create 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.
+
+* Support `opt(key, long)` and `must(key, long)`. 
+* Declare that `withFileStatus(null)` is allowed.
+* Declare that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+
+
+###  <a name="openfile(path)"></a> `FSDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `open(Path, int)`.
+
+Thus the chain `openFile(path).build().get()` has the same preconditions
+and postconditions as `open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of: 
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+The `openFile()` operation may check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and
+the actual `build()` and `get()` operations, this file-specific
+preconditions (file exists, file is readable, etc) MUST NOT be checked here.
+
+FileSystem implementations which do not implement `open(Path, int)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+### Implementors notes
+
+The base implementation of `openFileWithOptions()` actually executes
+the `open(path)` operation synchronously, yet still returns the result

Review comment:
       I think it calls open(path, int).

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create a builder to open a file, supporting options
+both standard and filesystem specific. The return

Review comment:
       "supporting both standard and filesystem specific options."

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create 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.
+
+* Support `opt(key, long)` and `must(key, long)`. 
+* Declare that `withFileStatus(null)` is allowed.
+* Declare that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+
+
+###  <a name="openfile(path)"></a> `FSDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `open(Path, int)`.
+
+Thus the chain `openFile(path).build().get()` has the same preconditions
+and postconditions as `open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of: 
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+The `openFile()` operation may check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and
+the actual `build()` and `get()` operations, this file-specific
+preconditions (file exists, file is readable, etc) MUST NOT be checked here.
+
+FileSystem implementations which do not implement `open(Path, int)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+### Implementors notes
+
+The base implementation of `openFileWithOptions()` actually executes
+the `open(path)` operation synchronously, yet still returns the result
+or any failures in the `CompletableFuture<>`, so as to ensure that users
+code expecting this.
+
+Any filesystem where the time to open a file may be significant SHOULD
+execute it asynchronously by submitting the operation in some executor/thread
+pool. This is particularly recommended for object stores and other filesystems
+likely to be accessed over long-haul connections.
+
+Arbitrary filesystem-specific options MAY be supported; these MUST
+be prefixed with either the filesystem schema, e.g. `hdfs.`
+or in the `fs.SCHEMA` format as normal configuration settings `fs.hdfs`. The
+latter style allows the same configuration option to be used for both
+filesystem configuration and file-specific configuration.
+
+It SHOULD be possible to always open a file without specifying any options,
+so as to present a consistent model to users. However, an implementation MAY
+opt to require one or more mandatory options to be set.
+
+The returned stream may perform "lazy" evaluation of file access. This is
+relevant for object stores where the probes for existence are expensive, and,
+even with an asynchronous open, may be considered needless.
+ 
+### <a name="openfile(pathhandle)"></a> `FSDataInputStreamBuilder 
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/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:
       What are mandatory keys, why are they mandatory and different from 
optional keys?  Are we using optional keys anywhere in this diff (sorry if I 
missed it)?

##########
File path: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
##########
@@ -522,32 +523,31 @@ private Constants() {
    * reading data.
    * Value: {@value}
    */
-  @InterfaceStability.Unstable
   public static final String INPUT_FADVISE =
       "fs.s3a.experimental.input.fadvise";

Review comment:
       I'm not clear on why this exists.  Seems S3 should be using the constant 
for "fs.option.openfile.fadvise" defined in OpenFileOptions.  Similar comment 
regarding the constants below.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
##########
@@ -518,4 +522,94 @@ 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/Options.java
##########
@@ -518,4 +522,94 @@ 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 {

Review comment:
       It may be time to break this file out into multiple files, each with one 
class.

##########
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.

Review comment:
       Copy a file or directory between filesystems.

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create 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.
+
+* Support `opt(key, long)` and `must(key, long)`. 
+* Declare that `withFileStatus(null)` is allowed.
+* Declare that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+
+
+###  <a name="openfile(path)"></a> `FSDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `open(Path, int)`.
+
+Thus the chain `openFile(path).build().get()` has the same preconditions
+and postconditions as `open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of: 
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+The `openFile()` operation may check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and
+the actual `build()` and `get()` operations, this file-specific
+preconditions (file exists, file is readable, etc) MUST NOT be checked here.
+
+FileSystem implementations which do not implement `open(Path, int)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+### Implementors notes
+
+The base implementation of `openFileWithOptions()` actually executes
+the `open(path)` operation synchronously, yet still returns the result
+or any failures in the `CompletableFuture<>`, so as to ensure that users
+code expecting this.
+
+Any filesystem where the time to open a file may be significant SHOULD
+execute it asynchronously by submitting the operation in some executor/thread
+pool. This is particularly recommended for object stores and other filesystems
+likely to be accessed over long-haul connections.
+
+Arbitrary filesystem-specific options MAY be supported; these MUST
+be prefixed with either the filesystem schema, e.g. `hdfs.`
+or in the `fs.SCHEMA` format as normal configuration settings `fs.hdfs`. The
+latter style allows the same configuration option to be used for both
+filesystem configuration and file-specific configuration.

Review comment:
       Did this already go public without "fs." prefix? Just wondering why we 
support both "fs.SCHEMA" and "SCHEMA".

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create a builder to open a file, supporting options
+both standard and filesystem specific. The return

Review comment:
       Can you also explain how standard and filesystem specific options are 
passed into the builder, or indicate that this is covered below?

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create 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.
+
+* Support `opt(key, long)` and `must(key, long)`. 
+* Declare that `withFileStatus(null)` is allowed.
+* Declare that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+
+
+###  <a name="openfile(path)"></a> `FSDataInputStreamBuilder openFile(Path 
path)`

Review comment:
       This returns a FutureDataInputStreamBuilder.  Also below, you should 
refer to the FutureDataInputStreamBuilder interface and not the private 
implementation (FsDataInputStreamBuilder ).

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create 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.
+
+* Support `opt(key, long)` and `must(key, long)`. 
+* Declare that `withFileStatus(null)` is allowed.
+* Declare that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+
+
+###  <a name="openfile(path)"></a> `FSDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `open(Path, int)`.

Review comment:
       I'm not familiar with the API and have to cross-reference with the 
source code.  I think this will be easier to read and understand if you include 
class names with the method names.

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create 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.
+
+* Support `opt(key, long)` and `must(key, long)`. 
+* Declare that `withFileStatus(null)` is allowed.
+* Declare that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+
+
+###  <a name="openfile(path)"></a> `FSDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `open(Path, int)`.
+
+Thus the chain `openFile(path).build().get()` has the same preconditions
+and postconditions as `open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of: 
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+The `openFile()` operation may check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and
+the actual `build()` and `get()` operations, this file-specific
+preconditions (file exists, file is readable, etc) MUST NOT be checked here.
+
+FileSystem implementations which do not implement `open(Path, int)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+### Implementors notes
+
+The base implementation of `openFileWithOptions()` actually executes
+the `open(path)` operation synchronously, yet still returns the result
+or any failures in the `CompletableFuture<>`, so as to ensure that users
+code expecting this.
+
+Any filesystem where the time to open a file may be significant SHOULD
+execute it asynchronously by submitting the operation in some executor/thread
+pool. This is particularly recommended for object stores and other filesystems
+likely to be accessed over long-haul connections.
+
+Arbitrary filesystem-specific options MAY be supported; these MUST
+be prefixed with either the filesystem schema, e.g. `hdfs.`
+or in the `fs.SCHEMA` format as normal configuration settings `fs.hdfs`. The
+latter style allows the same configuration option to be used for both
+filesystem configuration and file-specific configuration.
+
+It SHOULD be possible to always open a file without specifying any options,
+so as to present a consistent model to users. However, an implementation MAY
+opt to require one or more mandatory options to be set.
+
+The returned stream may perform "lazy" evaluation of file access. This is
+relevant for object stores where the probes for existence are expensive, and,
+even with an asynchronous open, may be considered needless.
+ 
+### <a name="openfile(pathhandle)"></a> `FSDataInputStreamBuilder 
openFile(PathHandle)`
+
+Creates a `FSDataInputStreamBuilder` to build an operation to open a file.
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file identified by the given `PathHandle` 
for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(PathHandle, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)` method
+returns a future which invokes `open(Path, int)`.
+
+Thus the chain `openFile(pathhandle).build().get()` has the same preconditions
+and postconditions as `open(Pathhandle, int)`
+
+As with `FSDataInputStreamBuilder openFile(PathHandle)`, the `openFile()`
+call must not be where path-specific preconditions are checked -that
+is postponed to the `build()` and `get()` calls.
+
+FileSystem implementations which do not implement `open(PathHandle handle, int 
bufferSize)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+The base implementation raises this exception in the `build()` operation;
+other implementations SHOULD copy this.
+
+
+## <a name="options"></a> Standard `openFile()` options

Review comment:
       1) Standard options gets its own section, so seems like filesystem 
specific options should too.  Or combine into a section that covers both types 
of options.
   
   2) I'm also thinking it is not a good idea to support filesystem specific 
options, but assumed there is a good reason for this.  It is nice when you can 
write code that can run on any filesystem.

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create 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.
+
+* Support `opt(key, long)` and `must(key, long)`. 
+* Declare that `withFileStatus(null)` is allowed.
+* Declare that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+
+
+###  <a name="openfile(path)"></a> `FSDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `open(Path, int)`.
+
+Thus the chain `openFile(path).build().get()` has the same preconditions
+and postconditions as `open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of: 
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+The `openFile()` operation may check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and
+the actual `build()` and `get()` operations, this file-specific
+preconditions (file exists, file is readable, etc) MUST NOT be checked here.
+
+FileSystem implementations which do not implement `open(Path, int)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+### Implementors notes
+
+The base implementation of `openFileWithOptions()` actually executes
+the `open(path)` operation synchronously, yet still returns the result
+or any failures in the `CompletableFuture<>`, so as to ensure that users
+code expecting this.
+
+Any filesystem where the time to open a file may be significant SHOULD
+execute it asynchronously by submitting the operation in some executor/thread
+pool. This is particularly recommended for object stores and other filesystems
+likely to be accessed over long-haul connections.
+
+Arbitrary filesystem-specific options MAY be supported; these MUST
+be prefixed with either the filesystem schema, e.g. `hdfs.`
+or in the `fs.SCHEMA` format as normal configuration settings `fs.hdfs`. The
+latter style allows the same configuration option to be used for both
+filesystem configuration and file-specific configuration.
+
+It SHOULD be possible to always open a file without specifying any options,
+so as to present a consistent model to users. However, an implementation MAY
+opt to require one or more mandatory options to be set.
+
+The returned stream may perform "lazy" evaluation of file access. This is
+relevant for object stores where the probes for existence are expensive, and,
+even with an asynchronous open, may be considered needless.
+ 
+### <a name="openfile(pathhandle)"></a> `FSDataInputStreamBuilder 
openFile(PathHandle)`
+
+Creates a `FSDataInputStreamBuilder` to build an operation to open a file.
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file identified by the given `PathHandle` 
for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(PathHandle, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)` method
+returns a future which invokes `open(Path, int)`.
+
+Thus the chain `openFile(pathhandle).build().get()` has the same preconditions
+and postconditions as `open(Pathhandle, int)`
+
+As with `FSDataInputStreamBuilder openFile(PathHandle)`, the `openFile()`
+call must not be where path-specific preconditions are checked -that
+is postponed to the `build()` and `get()` calls.
+
+FileSystem implementations which do not implement `open(PathHandle handle, int 
bufferSize)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+The base implementation raises this exception in the `build()` operation;
+other implementations SHOULD copy this.
+
+
+## <a name="options"></a> Standard `openFile()` options
+
+The standard `openFile()` options are defined in 
`org.apache.hadoop.fs.OpenFileOptions`;

Review comment:
       It's not clear how OpenFileOptions is used by the new OpenFile methods.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
##########
@@ -518,4 +522,94 @@ 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 seek policies: {@value}.

Review comment:
       I believe you changed the terminology.  "Seek policies" should be 
replaced with the new nomenclature, not only here, but everywhere.  I'm ok with 
any nomenclature, but it helps to be consistent, so "read strategy", "read 
mode", or "read policy" all sound good to me.

##########
File path: 
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md
##########
@@ -0,0 +1,290 @@
+<!---
+  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.
+-->
+
+# `openFile()`
+
+Create 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.
+
+* Support `opt(key, long)` and `must(key, long)`. 
+* Declare that `withFileStatus(null)` is allowed.
+* Declare that `withFileStatus(status)` only checks
+  the filename of the path, not the full path.
+  This is needed to support passthrough/mounted filesystems.
+
+
+###  <a name="openfile(path)"></a> `FSDataInputStreamBuilder openFile(Path 
path)`
+
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file at `path` for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(Path, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)`
+ultimately invokes `open(Path, int)`.
+
+Thus the chain `openFile(path).build().get()` has the same preconditions
+and postconditions as `open(Path p, int bufferSize)`
+
+However, there is one difference which implementations are free to
+take advantage of: 
+
+The returned stream MAY implement a lazy open where file non-existence or
+access permission failures may not surface until the first `read()` of the
+actual data.
+
+The `openFile()` operation may check the state of the filesystem during its
+invocation, but as the state of the filesystem may change betwen this call and
+the actual `build()` and `get()` operations, this file-specific
+preconditions (file exists, file is readable, etc) MUST NOT be checked here.
+
+FileSystem implementations which do not implement `open(Path, int)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+### Implementors notes
+
+The base implementation of `openFileWithOptions()` actually executes
+the `open(path)` operation synchronously, yet still returns the result
+or any failures in the `CompletableFuture<>`, so as to ensure that users
+code expecting this.
+
+Any filesystem where the time to open a file may be significant SHOULD
+execute it asynchronously by submitting the operation in some executor/thread
+pool. This is particularly recommended for object stores and other filesystems
+likely to be accessed over long-haul connections.
+
+Arbitrary filesystem-specific options MAY be supported; these MUST
+be prefixed with either the filesystem schema, e.g. `hdfs.`
+or in the `fs.SCHEMA` format as normal configuration settings `fs.hdfs`. The
+latter style allows the same configuration option to be used for both
+filesystem configuration and file-specific configuration.
+
+It SHOULD be possible to always open a file without specifying any options,
+so as to present a consistent model to users. However, an implementation MAY
+opt to require one or more mandatory options to be set.
+
+The returned stream may perform "lazy" evaluation of file access. This is
+relevant for object stores where the probes for existence are expensive, and,
+even with an asynchronous open, may be considered needless.
+ 
+### <a name="openfile(pathhandle)"></a> `FSDataInputStreamBuilder 
openFile(PathHandle)`
+
+Creates a `FSDataInputStreamBuilder` to build an operation to open a file.
+Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
+to construct a operation to open the file identified by the given `PathHandle` 
for reading.
+
+When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
+the builder parameters are verified and
+`openFileWithOptions(PathHandle, OpenFileParameters)` invoked.
+
+This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
+which, when its `get()` method is called, either returns an input
+stream of the contents of opened file, or raises an exception.
+
+The base implementation of the `openFileWithOptions(PathHandle, 
OpenFileParameters)` method
+returns a future which invokes `open(Path, int)`.
+
+Thus the chain `openFile(pathhandle).build().get()` has the same preconditions
+and postconditions as `open(Pathhandle, int)`
+
+As with `FSDataInputStreamBuilder openFile(PathHandle)`, the `openFile()`
+call must not be where path-specific preconditions are checked -that
+is postponed to the `build()` and `get()` calls.
+
+FileSystem implementations which do not implement `open(PathHandle handle, int 
bufferSize)`
+MAY postpone raising an `UnsupportedOperationException` until either the
+`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
+else they MAY fail fast in the `openFile()` call.
+
+The base implementation raises this exception in the `build()` operation;
+other implementations SHOULD copy this.
+
+
+## <a name="options"></a> Standard `openFile()` options
+
+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 fadvise or file length values and use them when opening files.
+
+Unless otherwise stated, they SHOULD be viewed as hints.
+
+#### `fs.option.openfile.buffer.size`
+
+Buffer size in bytes.
+
+This overrides the default value set in the configuration with the option
+`io.file.buffer.size`.
+
+It is supported by all filesystems which allow for stream-specific buffer sizes
+to be set via `open(path, buffersize)`.
+
+#### `fs.option.openfile.fadvise`

Review comment:
       The name "fadvise" does not quite fit how this is used.  Perhaps 
"readstrategy" or "readmode" would be better, and where you refer to "policy" 
call it "strategy" or "mode" instead.

##########
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/ChecksumFileSystem.java
##########
@@ -847,7 +847,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:
       Seems ChecksumFileSystem should not override the base class 
implementation, since it has the same implementation as the base class.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
##########
@@ -518,4 +522,94 @@ 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 seek policies: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE =
+        FS_OPTION_OPENFILE + "fadvise";
+
+    /**
+     * fadvise policy 'normal' -up to implementation: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE_NORMAL =

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

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
##########
@@ -518,4 +522,94 @@ 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 seek policies: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE =
+        FS_OPTION_OPENFILE + "fadvise";
+
+    /**
+     * fadvise policy 'normal' -up to implementation: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE_NORMAL =
+        "normal";
+
+    /**
+     * fadvise policy for sequential IO: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL =
+        "sequential";
+
+    /**
+     * fadvise policy for random: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE_RANDOM =
+        "random";
+
+    /**
+     * fadvise policy for adaptive IO: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE_ADAPTIVE =

Review comment:
       Why do we need adaptive?  If the caller requests sequential, random, or 
normal, I think that will suffice.

##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
##########
@@ -518,4 +522,94 @@ 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 seek policies: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE =
+        FS_OPTION_OPENFILE + "fadvise";
+
+    /**
+     * fadvise policy 'normal' -up to implementation: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE_NORMAL =
+        "normal";
+
+    /**
+     * fadvise policy for sequential IO: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL =
+        "sequential";
+
+    /**
+     * fadvise policy for random: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE_RANDOM =
+        "random";
+
+    /**
+     * fadvise policy for adaptive IO: {@value}.
+     */
+    public static final String FS_OPTION_OPENFILE_FADVISE_ADAPTIVE =
+        "adaptive";
+
+    /**
+     * Set of standard options which openFile implementations
+     * MUST recognize, even if they ignore the actual values.

Review comment:
       See my comment in the builder about possibly making these properties 
(with getter and setter).

##########
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/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.
+   * @throws IOException failure

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/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.

Review comment:
       The comment says do sequential reads, but then the sequential option 
isn't passed below.

##########
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_FADVISE_NORMAL)) {

Review comment:
       Isn't this the same as calling 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/impl/AbstractFSBuilderImpl.java
##########
@@ -46,7 +46,7 @@
  * <code>
  *   .opt("foofs:option.a", true)
  *   .opt("foofs:option.b", "value")
- *   .opt("barfs:cache", true)
+ *   .opt("fs.s3a.open.option.etag", "9fe4c37c25b")

Review comment:
       Why isn't etag also a standard option?  The less fs specific stuff the 
better, right?

##########
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:
       Should mandatory parameters be strongly typed fields on this class?  I 
think it would be better than a Set of strings.




----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 522024)
    Time Spent: 4h 10m  (was: 4h)

> 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: 4h 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: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to