[
https://issues.apache.org/jira/browse/HADOOP-16202?focusedWorklogId=754873&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-754873
]
ASF GitHub Bot logged work on HADOOP-16202:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 09/Apr/22 01:32
Start Date: 09/Apr/22 01:32
Worklog Time Spent: 10m
Work Description: mukund-thakur commented on code in PR #2584:
URL: https://github.com/apache/hadoop/pull/2584#discussion_r846483461
##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java:
##########
@@ -4751,9 +4752,13 @@ protected FSDataInputStreamBuilder(
Optional<Path> optionalPath = getOptionalPath();
OpenFileParameters parameters = new OpenFileParameters()
.withMandatoryKeys(getMandatoryKeys())
+ .withOptionalKeys(getOptionalKeys())
.withOptions(getOptions())
- .withBufferSize(getBufferSize())
.withStatus(super.getStatus()); // explicit to avoid IDE warnings
+ // buffer size can be configured
+ parameters.withBufferSize(
Review Comment:
Same as FileContext. why not in one go? What IDE warnings we are getting?
##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java:
##########
@@ -2930,9 +2940,13 @@ protected FSDataInputStreamBuilder(
final Path absF = fixRelativePart(getPath());
OpenFileParameters parameters = new OpenFileParameters()
.withMandatoryKeys(getMandatoryKeys())
+ .withOptionalKeys(getOptionalKeys())
.withOptions(getOptions())
.withBufferSize(getBufferSize())
.withStatus(getStatus());
+ parameters.withBufferSize(
Review Comment:
why not added just above but called separately?
##########
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md:
##########
@@ -65,37 +65,97 @@ 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 read policy of `random` has been specified,
+with the requirement that the filesystem implementation must understand the
option.
+An http-specific option has been supplied which may be interpreted by any
store;
+If the filesystem opening the file does not recognize the option, it can
safely be
+ignored.
+
+### When to use `opt()` versus `must()`
+
+The difference between `opt()` versus `must()` is how the FileSystem opening
+the file must react to an option which it does not recognize.
+
+* `opt(name, value)`: if `name` is not reco
Review Comment:
I think some typo here at the end.
##########
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md:
##########
@@ -128,26 +193,499 @@ completed, returns an input stream which can read data
from the filesystem.
The `build()` operation MAY perform the validation of the file's existence,
its kind, so rejecting attempts to read from a directory or non-existent
-file. **Alternatively**, the `build()` operation may delay all checks
-until an asynchronous operation whose outcome is provided by the `Future`
+file. Alternatively
+* file existence/status checks MAY be performed asynchronously within the
returned
+ `CompletableFuture<>`.
+* file existence/status checks MAY be postponed until the first byte is read in
+ any of the read such as `read()` or `PositionedRead`.
That is, the precondition `exists(FS, path)` and `isFile(FS, path)` are
-only guaranteed to have been met after the `get()` on the returned future is
successful.
+only guaranteed to have been met after the `get()` called on returned future
+and an attempt has been made to read the stream.
-Thus, if even a file does not exist, the following call will still succeed,
returning
-a future to be evaluated.
+Thus, if even when file does not exist, or is a directory rather than a file,
+the following call MUST succeed, returning a `CompletableFuture` to be
evaluated.
```java
Path p = new Path("file://tmp/file-which-does-not-exist");
CompletableFuture<FSDataInputStream> future = p.getFileSystem(conf)
.openFile(p)
- .build;
+ .build();
+```
+
+The inability to access/read a file MUST raise an `IOException`or subclass
+in either the future's `get()` call, or, for late binding operations,
+when an operation to read data is invoked.
+
+Therefore the following sequence SHALL fail when invoked on the
+`future` returned by the previous example.
+
+```java
+ future.get().read();
+```
+
+Access permission checks have the same visibility requirements: permission
failures
+MUST be delayed until the `get()` call and MAY be delayed into subsequent
operations.
+
+Note: some operations on the input stream, such as `seek()` may not attempt
any IO
+at all. Such operations MAY NOT raise exceotions when interacting with
+nonexistent/unreadable files.
+
+## <a name="options"></a> Standard `openFile()` options since Hadoop 3.3.3
+
+These are options which `FileSystem` and `FileContext` implementation
+MUST expected to recognise and MAY support by changing the behavior of
+their input streams as a appropriate.
Review Comment:
type: remove "a"
##########
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md:
##########
@@ -128,26 +193,499 @@ completed, returns an input stream which can read data
from the filesystem.
The `build()` operation MAY perform the validation of the file's existence,
its kind, so rejecting attempts to read from a directory or non-existent
-file. **Alternatively**, the `build()` operation may delay all checks
-until an asynchronous operation whose outcome is provided by the `Future`
+file. Alternatively
+* file existence/status checks MAY be performed asynchronously within the
returned
+ `CompletableFuture<>`.
+* file existence/status checks MAY be postponed until the first byte is read in
+ any of the read such as `read()` or `PositionedRead`.
That is, the precondition `exists(FS, path)` and `isFile(FS, path)` are
-only guaranteed to have been met after the `get()` on the returned future is
successful.
+only guaranteed to have been met after the `get()` called on returned future
+and an attempt has been made to read the stream.
-Thus, if even a file does not exist, the following call will still succeed,
returning
-a future to be evaluated.
+Thus, if even when file does not exist, or is a directory rather than a file,
+the following call MUST succeed, returning a `CompletableFuture` to be
evaluated.
```java
Path p = new Path("file://tmp/file-which-does-not-exist");
CompletableFuture<FSDataInputStream> future = p.getFileSystem(conf)
.openFile(p)
- .build;
+ .build();
+```
+
+The inability to access/read a file MUST raise an `IOException`or subclass
+in either the future's `get()` call, or, for late binding operations,
+when an operation to read data is invoked.
+
+Therefore the following sequence SHALL fail when invoked on the
+`future` returned by the previous example.
+
+```java
+ future.get().read();
+```
+
+Access permission checks have the same visibility requirements: permission
failures
+MUST be delayed until the `get()` call and MAY be delayed into subsequent
operations.
+
+Note: some operations on the input stream, such as `seek()` may not attempt
any IO
+at all. Such operations MAY NOT raise exceotions when interacting with
+nonexistent/unreadable files.
+
+## <a name="options"></a> Standard `openFile()` options since Hadoop 3.3.3
+
+These are options which `FileSystem` and `FileContext` implementation
+MUST expected to recognise and MAY support by changing the behavior of
+their input streams as a appropriate.
+
+Hadoop 3.3.0 added the `openFile()` API; these standard options were defined in
+a later release. Therefore, although they are "well known", unless confident
that
+the application will only be executed against releases of Hadoop which knows of
+the options -applications SHOULD 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 a `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
Review Comment:
typo: two "the" at the end.
##########
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md:
##########
@@ -128,26 +193,499 @@ completed, returns an input stream which can read data
from the filesystem.
The `build()` operation MAY perform the validation of the file's existence,
its kind, so rejecting attempts to read from a directory or non-existent
-file. **Alternatively**, the `build()` operation may delay all checks
-until an asynchronous operation whose outcome is provided by the `Future`
+file. Alternatively
+* file existence/status checks MAY be performed asynchronously within the
returned
+ `CompletableFuture<>`.
+* file existence/status checks MAY be postponed until the first byte is read in
+ any of the read such as `read()` or `PositionedRead`.
That is, the precondition `exists(FS, path)` and `isFile(FS, path)` are
-only guaranteed to have been met after the `get()` on the returned future is
successful.
+only guaranteed to have been met after the `get()` called on returned future
+and an attempt has been made to read the stream.
-Thus, if even a file does not exist, the following call will still succeed,
returning
-a future to be evaluated.
+Thus, if even when file does not exist, or is a directory rather than a file,
+the following call MUST succeed, returning a `CompletableFuture` to be
evaluated.
```java
Path p = new Path("file://tmp/file-which-does-not-exist");
CompletableFuture<FSDataInputStream> future = p.getFileSystem(conf)
.openFile(p)
- .build;
+ .build();
+```
+
+The inability to access/read a file MUST raise an `IOException`or subclass
+in either the future's `get()` call, or, for late binding operations,
+when an operation to read data is invoked.
+
+Therefore the following sequence SHALL fail when invoked on the
+`future` returned by the previous example.
+
+```java
+ future.get().read();
+```
+
+Access permission checks have the same visibility requirements: permission
failures
+MUST be delayed until the `get()` call and MAY be delayed into subsequent
operations.
+
+Note: some operations on the input stream, such as `seek()` may not attempt
any IO
+at all. Such operations MAY NOT raise exceotions when interacting with
+nonexistent/unreadable files.
+
+## <a name="options"></a> Standard `openFile()` options since Hadoop 3.3.3
+
+These are options which `FileSystem` and `FileContext` implementation
+MUST expected to recognise and MAY support by changing the behavior of
Review Comment:
typo: MUST be. Why are we putting a MUST recognise constraint..as it is a
new api altogether?
##########
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md:
##########
@@ -128,26 +193,499 @@ completed, returns an input stream which can read data
from the filesystem.
The `build()` operation MAY perform the validation of the file's existence,
its kind, so rejecting attempts to read from a directory or non-existent
-file. **Alternatively**, the `build()` operation may delay all checks
-until an asynchronous operation whose outcome is provided by the `Future`
+file. Alternatively
+* file existence/status checks MAY be performed asynchronously within the
returned
+ `CompletableFuture<>`.
+* file existence/status checks MAY be postponed until the first byte is read in
+ any of the read such as `read()` or `PositionedRead`.
That is, the precondition `exists(FS, path)` and `isFile(FS, path)` are
-only guaranteed to have been met after the `get()` on the returned future is
successful.
+only guaranteed to have been met after the `get()` called on returned future
+and an attempt has been made to read the stream.
-Thus, if even a file does not exist, the following call will still succeed,
returning
-a future to be evaluated.
+Thus, if even when file does not exist, or is a directory rather than a file,
+the following call MUST succeed, returning a `CompletableFuture` to be
evaluated.
```java
Path p = new Path("file://tmp/file-which-does-not-exist");
CompletableFuture<FSDataInputStream> future = p.getFileSystem(conf)
.openFile(p)
- .build;
+ .build();
+```
+
+The inability to access/read a file MUST raise an `IOException`or subclass
+in either the future's `get()` call, or, for late binding operations,
+when an operation to read data is invoked.
+
+Therefore the following sequence SHALL fail when invoked on the
+`future` returned by the previous example.
+
+```java
+ future.get().read();
+```
+
+Access permission checks have the same visibility requirements: permission
failures
+MUST be delayed until the `get()` call and MAY be delayed into subsequent
operations.
+
+Note: some operations on the input stream, such as `seek()` may not attempt
any IO
+at all. Such operations MAY NOT raise exceotions when interacting with
+nonexistent/unreadable files.
+
+## <a name="options"></a> Standard `openFile()` options since Hadoop 3.3.3
+
+These are options which `FileSystem` and `FileContext` implementation
+MUST expected to recognise and MAY support by changing the behavior of
+their input streams as a appropriate.
+
+Hadoop 3.3.0 added the `openFile()` API; these standard options were defined in
+a later release. Therefore, although they are "well known", unless confident
that
+the application will only be executed against releases of Hadoop which knows of
+the options -applications SHOULD 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 a `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.
+
+Unless otherwise stated, they SHOULD be viewed as hints.
+
+Note: if a standard option is added such that if set but not
+supported would be an error, then implementations SHALL reject it. For example,
+the S3A filesystem client supports the ability to push down SQL commands. If
+something like that were ever standardized, then the use of the option, either
+in `opt()` or `must()` argument MUST be rejected for filesystems which don't
+support the feature.
+
+### <a name="buffer.size"></a> Option: `fs.option.openfile.buffer.size`
+
+Read 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 filesystem clients which allow for stream-specific
buffer
+sizes to be set via `FileSystem.open(path, buffersize)`.
+
+### <a name="read.policy"></a> Option: `fs.option.openfile.read.policy`
+
+Declare the read policy of the input stream. This is a hint as to what the
+expected read pattern of an input stream will be. This MAY control readahead,
+buffering and other optimizations.
+
+Sequential reads may be optimized with prefetching data and/or reading data in
+larger blocks. Some applications (e.g. distCp) perform sequential IO even over
+columnar data.
+
+In contrast, random IO reads data in different parts of the file using a
+sequence of `seek()/read()`
+or via the `PositionedReadable` or `ByteBufferPositionedReadable` APIs.
+
+Random IO performance may be best if little/no prefetching takes place, along
+with other possible optimizations
+
+Queries over columnar formats such as Apache ORC and Apache Parquet perform
such
+random IO; other data formats may be best read with sequential or whole-file
+policies.
+
+What is key is that optimizing reads for seqential reads may impair random
+performance -and vice versa.
Review Comment:
typo: "-and"
Issue Time Tracking
-------------------
Worklog Id: (was: 754873)
Time Spent: 17h 10m (was: 17h)
> Enhance openFile() for better read performance against object stores
> ---------------------------------------------------------------------
>
> Key: HADOOP-16202
> URL: https://issues.apache.org/jira/browse/HADOOP-16202
> Project: Hadoop Common
> Issue Type: Bug
> 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: 17h 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.20.1#820001)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]