|
@@ -25,6 +25,55 @@ references to `FSDataInputStream` and its subclasses.
|
|
|
It is used to initate a (potentially asynchronous) operation to open an existing
|
|
|
file for reading.
|
|
|
|
|
|
+
|
|
|
+## <a name="History"></a> History
|
|
|
+
|
|
|
+### Hadoop 3.3.0: API introduced
|
|
|
+
|
|
|
+[HADOOP-15229](https://issues.apache.org/jira/browse/HADOOP-15229)
|
|
|
+_Add FileSystem builder-based openFile() API to match createFile()_
|
|
|
+
|
|
|
+* No `opt(String key, long value)` method was available.
|
|
|
+* the `withFileStatus(status)` call required a non-null parameter.
|
|
|
+* Sole Filesystem to process options and file status was S3A;
|
|
|
+* Only the s3a specific options were the S3 select and `fs.s3a.experimental.input.fadvise`
|
|
|
+* S3A Filesystem raised `IllegalArgumentException` if a file status was passed in
|
|
|
+ and the path of the filestatus did not match the path of the `openFile(path)` call.
|
|
|
+
|
|
|
+This is the baseline implementation. To write code guaranteed to compile against this version,
|
|
|
+use the `opt(String, String)` and `must(String, String)` methods, converting numbers to
|
|
|
+string explicitly.
|
|
|
+
|
|
|
+```java
|
|
|
+fs.open("s3a://bucket/file")
|
|
|
+ .opt("fs.option.openfile.length", Long.toString(length))
|
|
|
+ .build().get()
|
|
|
+```
|
|
|
+
|
|
|
+### Hadoop 3.3.5: standardization and expansion
|
|
|
+
|
|
|
+[HADOOP-16202](https://issues.apache.org/jira/browse/HADOOP-16202)
|
|
|
+_Enhance openFile() for better read performance against object stores_
|
|
|
+
|
|
|
+* `withFileStatus(null)` required to be accepted (and ignored)
|
|
|
+* only the filename part of any supplied FileStatus path must match the
|
|
|
+ filename passed in on `openFile(path)`.
|
|
|
+* An `opt(String key, long value)` option was added. *This is now deprecated as it
|
|
|
+caused regression
|
|
|
+* Standard `fs.option.openfile` options defined.
|
|
|
+* S3A FS to use openfile length option, seek start/end options not _yet_ used.
|
|
|
+* Azure ABFS connector takes a supplied `VersionedFileStatus` and omits any
|
|
|
+ HEAD probe for the object.
|
|
|
+
|
|
|
+### Hadoop 3.3.6: API change to address operator overload bugs.
|
|
|
+
|
|
|
+new `optLong()`, `optDouble()`, `mustLong()` and `mustDouble()` builder methods.
|
|
|
+
|
|
|
+* See [HADOOP-18724](https://issues.apache.org/jira/browse/HADOOP-18724) _Open file fails with NumberFormatException for S3AFileSystem_,
|
|
|
+ which was somehow caused by the overloaded `opt(long)`.
|
|
|
+* Specification updated to declare that unparseable numbers MUST be treated as "unset" and the default
|
|
|
+ value used instead.
|
|
|
+
|
|
|
## Invariants
|
|
|
|
|
|
The `FutureDataInputStreamBuilder` interface does not require parameters or
|
|
@@ -36,7 +85,7 @@ Some aspects of the state of the filesystem, MAY be checked in the initial
|
|
|
change between `openFile()` and the `build().get()` sequence. For example,
|
|
|
path validation.
|
|
|
|
|
|
-## Implementation-agnostic parameters.
|
|
|
+## <a name="parameters"></a> `Implementation-agnostic parameters.
|
|
|
|
|
|
|
|
|
### <a name="Builder.bufferSize"></a> `FutureDataInputStreamBuilder bufferSize(int bufSize)`
|
|
@@ -89,10 +138,20 @@ operations. This is to support wrapper filesystems and serialization/deserializa
|
|
|
of the status.
|
|
|
|
|
|
|
|
|
-### Set optional or mandatory parameters
|
|
|
+### <a name="optional"></a> Set optional or mandatory parameters
|
|
|
|
|
|
- FutureDataInputStreamBuilder opt(String key, ...)
|
|
|
- FutureDataInputStreamBuilder must(String key, ...)
|
|
|
+```java
|
|
|
+FutureDataInputStreamBuilder opt(String key, String value)
|
|
|
+FutureDataInputStreamBuilder opt(String key, int value)
|
|
|
+FutureDataInputStreamBuilder opt(String key, boolean value)
|
|
|
+FutureDataInputStreamBuilder optLong(String key, long value)
|
|
|
+FutureDataInputStreamBuilder optDouble(String key, double value)
|
|
|
+FutureDataInputStreamBuilder must(String key, String value)
|
|
|
+FutureDataInputStreamBuilder must(String key, int value)
|
|
|
+FutureDataInputStreamBuilder must(String key, boolean value)
|
|
|
+FutureDataInputStreamBuilder mustLong(String key, long value)
|
|
|
+FutureDataInputStreamBuilder mustDouble(String key, double value)
|
|
|
+```
|
|
|
|
|
|
Set optional or mandatory parameters to the builder. Using `opt()` or `must()`,
|
|
|
client can specify FS-specific parameters without inspecting the concrete type
|
|
@@ -103,7 +162,7 @@ Example:
|
|
|
```java
|
|
|
out = fs.openFile(path)
|
|
|
.must("fs.option.openfile.read.policy", "random")
|
|
|
- .opt("fs.http.connection.timeout", 30_000L)
|
|
|
+ .optLong("fs.http.connection.timeout", 30_000L)
|
|
|
.withFileStatus(statusFromListing)
|
|
|
.build()
|
|
|
.get();
|
|
@@ -115,9 +174,9 @@ 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()`
|
|
|
+### <a name="usage"></a> When to use `opt` versus `must`
|
|
|
|
|
|
-The difference between `opt()` versus `must()` is how the FileSystem opening
|
|
|
+The difference between `opt` versus `must` is how the FileSystem opening
|
|
|
the file must react to an option which it does not recognize.
|
|
|
|
|
|
```python
|
|
@@ -144,7 +203,7 @@ irrespective of how the (key, value) pair was declared.
|
|
|
defined in this filesystem specification, validated through contract
|
|
|
tests.
|
|
|
|
|
|
-#### Implementation Notes
|
|
|
+## <a name="implementation"></a> Implementation Notes
|
|
|
|
|
|
Checking for supported options must be performed in the `build()` operation.
|
|
|
|
|
@@ -155,6 +214,13 @@ Checking for supported options must be performed in the `build()` operation.
|
|
|
a feature which is recognized but not supported in the specific
|
|
|
`FileSystem`/`FileContext` instance `UnsupportedException` MUST be thrown.
|
|
|
|
|
|
+Parsing of numeric values SHOULD trim any string and if the value
|
|
|
+cannot be parsed as a number, downgrade to any default value supplied.
|
|
|
+This is to address [HADOOP-18724](https://issues.apache.org/jira/browse/HADOOP-18724)
|
|
|
+_Open file fails with NumberFormatException for S3AFileSystem_, which was cause by the overloaded `opt()`
|
|
|
+builder parameter binding to `opt(String, double)` rather than `opt(String, long)` when a long
|
|
|
+value was passed in.
|
|
|
+
|
|
|
The behavior of resolving the conflicts between the parameters set by
|
|
|
builder methods (i.e., `bufferSize()`) and `opt()`/`must()` is as follows:
|
|
|
|
|
@@ -181,7 +247,7 @@ Even if not values of the status are used, the presence of the argument
|
|
|
can be interpreted as the caller declaring that they believe the file
|
|
|
to be present and of the given size.
|
|
|
|
|
|
-## Builder interface
|
|
|
+## <a name="builder"></a> Builder interface
|
|
|
|
|
|
### <a name="build"></a> `CompletableFuture<FSDataInputStream> build()`
|
|
|
|
|
@@ -339,7 +405,7 @@ _Futher reading_
|
|
|
* [Linux fadvise()](https://linux.die.net/man/2/fadvise).
|
|
|
* [Windows `CreateFile()`](https://docs.microsoft.com/en-us/windows/win32/api/fileapi/nf-fileapi-createfilea#caching-behavior)
|
|
|
|
|
|
-#### <a name="read.policy."></a> Read Policy `adaptive`
|
|
|
+#### <a name="read.policy.adaptive"></a> Read Policy `adaptive`
|
|
|
|
|
|
Try to adapt the seek policy to the read pattern of the application.
|
|
|
|
|
@@ -429,7 +495,7 @@ If this option is used by the FileSystem implementation
|
|
|
|
|
|
*Implementor's Notes*
|
|
|
|
|
|
-* A value of `fs.option.openfile.length` < 0 MUST be rejected.
|
|
|
+* A value of `fs.option.openfile.length` < 0 MUST be ignored.
|
|
|
* If a file status is supplied along with a value in `fs.opt.openfile.length`;
|
|
|
the file status values take precedence.
|
|
|
|
|
@@ -466,11 +532,11 @@ than that value.
|
|
|
|
|
|
The S3A Connector supports custom options for readahead and seek policy.
|
|
|
|
|
|
-| Name | Type | Meaning |
|
|
|
-|--------------------------------------|----------|-------------------------------------------------------------|
|
|
|
-| `fs.s3a.readahead.range` | `long` | readahead range in bytes |
|
|
|
-| `fs.s3a.input.async.drain.threshold` | `long` | threshold to switch to asynchronous draining of the stream |
|
|
|
-| `fs.s3a.experimental.input.fadvise` | `String` | seek policy. Superceded by `fs.option.openfile.read.policy` |
|
|
|
+| Name | Type | Meaning |
|
|
|
+|--------------------------------------|----------|---------------------------------------------------------------------------|
|
|
|
+| `fs.s3a.readahead.range` | `long` | readahead range in bytes |
|
|
|
+| `fs.s3a.experimental.input.fadvise` | `String` | seek policy. Superceded by `fs.option.openfile.read.policy` |
|
|
|
+| `fs.s3a.input.async.drain.threshold` | `long` | threshold to switch to asynchronous draining of the stream. (Since 3.3.5) |
|
|
|
|
|
|
If the option set contains a SQL statement in the `fs.s3a.select.sql` statement,
|
|
|
then the file is opened as an S3 Select query.
|
|
@@ -510,8 +576,8 @@ protected SeekableInputStream newStream(Path path, FileStatus stat,
|
|
|
.opt("fs.option.openfile.read.policy", "vector, random")
|
|
|
.withFileStatus(stat);
|
|
|
|
|
|
- builder.opt("fs.option.openfile.split.start", splitStart);
|
|
|
- builder.opt("fs.option.openfile.split.end", splitEnd);
|
|
|
+ builder.optLong("fs.option.openfile.split.start", splitStart);
|
|
|
+ builder.optLong("fs.option.openfile.split.end", splitEnd);
|
|
|
CompletableFuture<FSDataInputStream> streamF = builder.build();
|
|
|
return HadoopStreams.wrap(FutureIO.awaitFuture(streamF));
|
|
|
}
|
|
@@ -618,8 +684,8 @@ An example of a record reader passing in options to the file it opens.
|
|
|
file.getFileSystem(job).openFile(file);
|
|
|
// the start and end of the split may be used to build
|
|
|
// an input strategy.
|
|
|
- builder.opt("fs.option.openfile.split.start", start);
|
|
|
- builder.opt("fs.option.openfile.split.end", end);
|
|
|
+ builder.optLong("fs.option.openfile.split.start", start);
|
|
|
+ builder.optLong("fs.option.openfile.split.end", end);
|
|
|
FutureIO.propagateOptions(builder, job,
|
|
|
"mapreduce.job.input.file.option",
|
|
|
"mapreduce.job.input.file.must");
|
|
@@ -633,7 +699,7 @@ An example of a record reader passing in options to the file it opens.
|
|
|
### `FileContext.openFile`
|
|
|
|
|
|
From `org.apache.hadoop.fs.AvroFSInput`; a file is opened with sequential input.
|
|
|
-Because the file length has already been probed for, the length is passd down
|
|
|
+Because the file length has already been probed for, the length is passed down
|
|
|
|
|
|
```java
|
|
|
public AvroFSInput(FileContext fc, Path p) throws IOException {
|
|
@@ -642,7 +708,7 @@ Because the file length has already been probed for, the length is passd down
|
|
|
this.stream = awaitFuture(fc.openFile(p)
|
|
|
.opt("fs.option.openfile.read.policy",
|
|
|
"sequential")
|
|
|
- .opt("fs.option.openfile.length",
|
|
|
+ .optLong("fs.option.openfile.length",
|
|
|
Long.toString(status.getLen()))
|
|
|
.build());
|
|
|
fc.open(p);
|
|
@@ -682,8 +748,3 @@ public T load(FileSystem fs,
|
|
|
}
|
|
|
}
|
|
|
```
|
|
|
-
|
|
|
-*Note:* : in Hadoop 3.3.2 and earlier, the `withFileStatus(status)` call
|
|
|
-required a non-null parameter; this has since been relaxed.
|
|
|
-For maximum compatibility across versions, only invoke the method
|
|
|
-when the file status is known to be non-null.
|