You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2022/04/01 15:47:18 UTC

[GitHub] [hadoop] steveloughran commented on a change in pull request #2584: HADOOP-16202. Enhance openFile() for better read performance against object stores

steveloughran commented on a change in pull request #2584:
URL: https://github.com/apache/hadoop/pull/2584#discussion_r840714093



##########
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:
       i'm trying to say that the read and length policies must be understood, but may not be used, if the implementation chooses not to




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

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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