Skip to content

Commit e7bf3e7

Browse files
committed
HADOOP-16202. openFile()
Name and document standard options; move to visible class definition. Document better in filesystem spec, including examples of API and MapReduce use Change-Id: Ia593a67babcad63c13dda65b62287683a678a266
1 parent 7617109 commit e7bf3e7

File tree

18 files changed

+447
-209
lines changed

18 files changed

+447
-209
lines changed

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4536,7 +4536,7 @@ protected CompletableFuture<FSDataInputStream> openFileWithOptions(
45364536
final OpenFileParameters parameters) throws IOException {
45374537
AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
45384538
parameters.getMandatoryKeys(),
4539-
OpenFileParameters.STANDARD_OPTIONS, "");
4539+
OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS, "");
45404540
CompletableFuture<FSDataInputStream> result = new CompletableFuture<>();
45414541
try {
45424542
result.complete(open(pathHandle, parameters.getBufferSize()));

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
*/
1818
package org.apache.hadoop.fs;
1919

20+
import javax.annotation.Nullable;
2021
import java.io.IOException;
2122
import java.util.concurrent.CompletableFuture;
2223

@@ -27,14 +28,14 @@
2728
* Builder for input streams and subclasses whose return value is
2829
* actually a completable future: this allows for better asynchronous
2930
* operation.
30-
*
31+
* <p></p>
3132
* To be more generic, {@link #opt(String, int)} and {@link #must(String, int)}
3233
* variants provide implementation-agnostic way to customize the builder.
3334
* Each FS-specific builder implementation can interpret the FS-specific
3435
* options accordingly, for example:
35-
*
36+
* <p></p>
3637
* If the option is not related to the file system, the option will be ignored.
37-
* If the option is must, but not supported by the file system, a
38+
* If the option is must, but not supported/known by the file system, an
3839
* {@link IllegalArgumentException} will be thrown.
3940
*
4041
*/
@@ -51,10 +52,11 @@ CompletableFuture<FSDataInputStream> build()
5152
/**
5253
* A FileStatus may be provided to the open request.
5354
* It is up to the implementation whether to use this or not.
54-
* @param status status.
55+
* @param status status: may be null
5556
* @return the builder.
5657
*/
57-
default FutureDataInputStreamBuilder withFileStatus(FileStatus status) {
58+
default FutureDataInputStreamBuilder withFileStatus(
59+
@Nullable FileStatus status) {
5860
return this;
5961
}
6062

Lines changed: 92 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,92 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hadoop.fs;
20+
21+
import java.util.Set;
22+
import java.util.stream.Collectors;
23+
import java.util.stream.Stream;
24+
25+
import org.apache.hadoop.classification.InterfaceAudience;
26+
import org.apache.hadoop.classification.InterfaceStability;
27+
28+
/**
29+
* The standard openfile options.
30+
*/
31+
@InterfaceAudience.Public
32+
@InterfaceStability.Evolving
33+
public class OpenFileOptions {
34+
35+
/**
36+
* Prefix for all standard filesystem options: {@value}.
37+
*/
38+
public static final String FILESYSTEM_OPTION = "fs.option.";
39+
40+
/**
41+
* Prefix for all openFile options: {@value}.
42+
*/
43+
public static final String FS_OPTION_OPENFILE =
44+
FILESYSTEM_OPTION + "openfile.";
45+
46+
/**
47+
* OpenFile option for seek policies: {@value}.
48+
*/
49+
public static final String FS_OPTION_OPENFILE_LENGTH =
50+
FS_OPTION_OPENFILE + "length";
51+
52+
/**
53+
* OpenFile option for seek policies: {@value}.
54+
*/
55+
public static final String FS_OPTION_OPENFILE_FADVISE =
56+
FS_OPTION_OPENFILE + "fadvise";
57+
58+
/**
59+
* fadvise policy: {@value}.
60+
*/
61+
public static final String FS_OPTION_OPENFILE_FADVISE_NORMAL =
62+
"normal";
63+
64+
/**
65+
* fadvise policy: {@value}.
66+
*/
67+
public static final String FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL =
68+
"sequential";
69+
70+
/**
71+
* fadvise policy: {@value}.
72+
*/
73+
public static final String FS_OPTION_OPENFILE_FADVISE_RANDOM =
74+
"random";
75+
76+
/**
77+
* fadvise policy: {@value}.
78+
*/
79+
public static final String FS_OPTION_OPENFILE_FADVISE_ADAPTIVE =
80+
"adaptive";
81+
82+
/**
83+
* Set of standard options which openfile implementations
84+
* MUST recognize, even if they ignore the actual values.
85+
*/
86+
public static final Set<String> FS_OPTION_OPENFILE_STANDARD_OPTIONS =
87+
Stream.of(
88+
FS_OPTION_OPENFILE_FADVISE,
89+
FS_OPTION_OPENFILE_LENGTH)
90+
.collect(Collectors.toSet());
91+
92+
}

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
package org.apache.hadoop.fs.impl;
2020

2121
import javax.annotation.Nonnull;
22+
import javax.annotation.Nullable;
2223
import java.io.IOException;
2324
import java.util.concurrent.CompletableFuture;
2425

@@ -40,16 +41,15 @@
4041
* Builder for input streams and subclasses whose return value is
4142
* actually a completable future: this allows for better asynchronous
4243
* operation.
43-
*
44+
* <p></p>
4445
* To be more generic, {@link #opt(String, int)} and {@link #must(String, int)}
4546
* variants provide implementation-agnostic way to customize the builder.
4647
* Each FS-specific builder implementation can interpret the FS-specific
4748
* options accordingly, for example:
48-
*
49+
* <p></p>
4950
* If the option is not related to the file system, the option will be ignored.
50-
* If the option is must, but not supported by the file system, a
51+
* If the option is must, but not supported/known by the file system, an
5152
* {@link IllegalArgumentException} will be thrown.
52-
*
5353
*/
5454
@InterfaceAudience.Public
5555
@InterfaceStability.Unstable
@@ -147,7 +147,8 @@ public FutureDataInputStreamBuilder getThisBuilder() {
147147
}
148148

149149
@Override
150-
public FutureDataInputStreamBuilder withFileStatus(FileStatus st) {
150+
public FutureDataInputStreamBuilder withFileStatus(
151+
@Nullable FileStatus st) {
151152
this.status = st;
152153
return this;
153154
}

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java

Lines changed: 0 additions & 43 deletions
Original file line numberDiff line numberDiff line change
@@ -106,47 +106,4 @@ public FileStatus getStatus() {
106106
return status;
107107
}
108108

109-
110-
/**
111-
* OpenFile option for seek policies: {@value}.
112-
*/
113-
public static final String FS_OPT_OPENFILE_FADVISE =
114-
"fs.opt.openfile.fadvise";
115-
116-
/**
117-
* fadvise policy: {@value}.
118-
*/
119-
public static final String FS_OPT_OPENFILE_FADVISE_NORMAL = "normal";
120-
121-
/**
122-
* fadvise policy: {@value}.
123-
*/
124-
public static final String FS_OPT_OPENFILE_FADVISE_SEQUENTIAL = "sequential";
125-
126-
/**
127-
* fadvise policy: {@value}.
128-
*/
129-
public static final String FS_OPT_OPENFILE_FADVISE_RANDOM = "random";
130-
131-
/**
132-
* fadvise policy: {@value}.
133-
*/
134-
public static final String FS_OPT_OPENFILE_FADVISE_ADAPTIVE = "adaptive";
135-
136-
/**
137-
* OpenFile option for seek policies: {@value}.
138-
*/
139-
public static final String FS_OPT_OPENFILE_LENGTH =
140-
"fs.opt.openfile.length";
141-
142-
/**
143-
* Set of standard options.
144-
*/
145-
public static final Set<String> STANDARD_OPTIONS =
146-
Stream.of(
147-
FS_OPT_OPENFILE_FADVISE,
148-
FS_OPT_OPENFILE_LENGTH)
149-
.collect(Collectors.toSet());
150-
151-
152109
}

hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md

Lines changed: 2 additions & 88 deletions
Original file line numberDiff line numberDiff line change
@@ -728,97 +728,11 @@ exists in the metadata, but no copies of any its blocks can be located;
728728

729729
### `FSDataInputStreamBuilder openFile(Path path)`
730730

731-
Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
732-
to construct a operation to open the file at `path` for reading.
731+
See [openFile()](openfile.html).
733732

734-
When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
735-
the builder parameters are verified and
736-
`openFileWithOptions(Path, OpenFileParameters)` invoked.
737-
738-
This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
739-
which, when its `get()` method is called, either returns an input
740-
stream of the contents of opened file, or raises an exception.
741-
742-
The base implementation of the `openFileWithOptions(PathHandle, OpenFileParameters)`
743-
ultimately invokes `open(Path, int)`.
744-
745-
Thus the chain `openFile(path).build().get()` has the same preconditions
746-
and postconditions as `open(Path p, int bufferSize)`
747-
748-
However, there is one difference which implementations are free to
749-
take advantage of:
750-
751-
The returned stream MAY implement a lazy open where file non-existence or
752-
access permission failures may not surface until the first `read()` of the
753-
actual data.
754-
755-
The `openFile()` operation may check the state of the filesystem during its
756-
invocation, but as the state of the filesystem may change betwen this call and
757-
the actual `build()` and `get()` operations, this file-specific
758-
preconditions (file exists, file is readable, etc) MUST NOT be checked here.
759-
760-
FileSystem implementations which do not implement `open(Path, int)`
761-
MAY postpone raising an `UnsupportedOperationException` until either the
762-
`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
763-
else they MAY fail fast in the `openFile()` call.
764-
765-
### Implementors notes
766-
767-
The base implementation of `openFileWithOptions()` actually executes
768-
the `open(path)` operation synchronously, yet still returns the result
769-
or any failures in the `CompletableFuture<>`, so as to ensure that users
770-
code expecting this.
771-
772-
Any filesystem where the time to open a file may be significant SHOULD
773-
execute it asynchronously by submitting the operation in some executor/thread
774-
pool. This is particularly recommended for object stores and other filesystems
775-
likely to be accessed over long-haul connections.
776-
777-
Arbitrary filesystem-specific options MAY be supported; these MUST
778-
be prefixed with either the filesystem schema, e.g. `hdfs.`
779-
or in the "fs.SCHEMA" format as normal configuration settings `fs.hdfs`). The
780-
latter style allows the same configuration option to be used for both
781-
filesystem configuration and file-specific configuration.
782-
783-
It SHOULD be possible to always open a file without specifying any options,
784-
so as to present a consistent model to users. However, an implementation MAY
785-
opt to require one or more mandatory options to be set.
786-
787-
The returned stream may perform "lazy" evaluation of file access. This is
788-
relevant for object stores where the probes for existence are expensive, and,
789-
even with an asynchronous open, may be considered needless.
790-
791733
### `FSDataInputStreamBuilder openFile(PathHandle)`
792734

793-
Creates a `FSDataInputStreamBuilder` to build an operation to open a file.
794-
Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html)
795-
to construct a operation to open the file identified by the given `PathHandle` for reading.
796-
797-
When `build()` is invoked on the returned `FSDataInputStreamBuilder` instance,
798-
the builder parameters are verified and
799-
`openFileWithOptions(PathHandle, OpenFileParameters)` invoked.
800-
801-
This (protected) operation returns a `CompletableFuture<FSDataInputStream>`
802-
which, when its `get()` method is called, either returns an input
803-
stream of the contents of opened file, or raises an exception.
804-
805-
The base implementation of the `openFileWithOptions(PathHandle, OpenFileParameters)` method
806-
returns a future which invokes `open(Path, int)`.
807-
808-
Thus the chain `openFile(pathhandle).build().get()` has the same preconditions
809-
and postconditions as `open(Pathhandle, int)`
810-
811-
As with `FSDataInputStreamBuilder openFile(PathHandle)`, the `openFile()`
812-
call must not be where path-specific preconditions are checked -that
813-
is postponed to the `build()` and `get()` calls.
814-
815-
FileSystem implementations which do not implement `open(PathHandle handle, int bufferSize)`
816-
MAY postpone raising an `UnsupportedOperationException` until either the
817-
`FSDataInputStreamBuilder.build()` or the subsequent `get()` call,
818-
else they MAY fail fast in the `openFile()` call.
819-
820-
The base implementation raises this exception in the `build()` operation;
821-
other implementations SHOULD copy this.
735+
See [openFile()](openfile.html).
822736

823737
### `PathHandle getPathHandle(FileStatus stat, HandleOpt... options)`
824738

hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -38,3 +38,5 @@ HDFS as these are commonly expected by Hadoop client applications.
3838
2. [Testing with the Filesystem specification](testing.html)
3939
2. [Extending the specification and its tests](extending.html)
4040
1. [Uploading a file using Multiple Parts](multipartuploader.html)
41+
1. [openFile()](openfile.html).
42+

0 commit comments

Comments
 (0)