From a408a4bae7995cd32bb876bade8b7e5dcd230f6d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 24 Jul 2020 13:42:25 +0100 Subject: [PATCH 01/10] HADOOP-16202. Enhance openFile() Roll-up of the previous PR Change-Id: Ib0aec173afcd8aae33f52da3f99ac813bd38c32f --- .../java/org/apache/hadoop/fs/FSBuilder.java | 14 + .../org/apache/hadoop/fs/FileContext.java | 1 + .../java/org/apache/hadoop/fs/FileSystem.java | 3 +- .../hadoop/fs/impl/AbstractFSBuilderImpl.java | 46 ++- .../FutureDataInputStreamBuilderImpl.java | 2 +- .../hadoop/fs/impl/OpenFileParameters.java | 58 ++++ .../filesystem/fsdatainputstreambuilder.md | 154 +++++++++- .../fs/contract/AbstractContractOpenTest.java | 24 +- .../hadoop/fs/contract/ContractTestUtils.java | 13 +- .../org/apache/hadoop/fs/s3a/Constants.java | 6 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 173 ++++------- .../apache/hadoop/fs/s3a/S3AInputPolicy.java | 2 + .../apache/hadoop/fs/s3a/S3AInputStream.java | 2 +- .../hadoop/fs/s3a/impl/InternalConstants.java | 21 +- .../hadoop/fs/s3a/impl/S3AOpenFileHelper.java | 287 ++++++++++++++++++ .../s3a/select/InternalSelectConstants.java | 2 +- .../fs/contract/s3a/ITestS3AContractOpen.java | 66 ++++ .../fs/s3a/ITestS3AFileOperationCost.java | 106 ++++++- .../fs/s3a/impl/TestOpenFileHelper.java | 250 +++++++++++++++ 19 files changed, 1092 insertions(+), 138 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileHelper.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java index b7757a62e28ad..a4c7254cfeb3c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java @@ -61,6 +61,13 @@ public interface FSBuilder> { */ 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); + /** * Set optional double parameter for the Builder. * @@ -104,6 +111,13 @@ public interface FSBuilder> { */ B must(@Nonnull String key, float value); + /** + * Set mandatory long option. + * + * @see #must(String, String) + */ + B must(@Nonnull String key, long value); + /** * Set mandatory double option. * diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java index 000c158d88c2e..32824b4824831 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java @@ -2936,6 +2936,7 @@ public CompletableFuture build() throws IOException { final Path absF = fixRelativePart(getPath()); OpenFileParameters parameters = new OpenFileParameters() .withMandatoryKeys(getMandatoryKeys()) + .withOptionalKeys(getOptionalKeys()) .withOptions(getOptions()) .withBufferSize(getBufferSize()) .withStatus(getStatus()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index e814b3da91536..e0f566d94b7c6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -4655,7 +4655,7 @@ protected CompletableFuture openFileWithOptions( final OpenFileParameters parameters) throws IOException { AbstractFSBuilderImpl.rejectUnknownMandatoryKeys( parameters.getMandatoryKeys(), - Collections.emptySet(), ""); + OpenFileParameters.STANDARD_OPTIONS, ""); CompletableFuture result = new CompletableFuture<>(); try { result.complete(open(pathHandle, parameters.getBufferSize())); @@ -4762,6 +4762,7 @@ public CompletableFuture build() throws IOException { Optional optionalPath = getOptionalPath(); OpenFileParameters parameters = new OpenFileParameters() .withMandatoryKeys(getMandatoryKeys()) + .withOptionalKeys(getOptionalKeys()) .withOptions(getOptions()) .withBufferSize(getBufferSize()) .withStatus(super.getStatus()); // explicit to avoid IDE warnings diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java index 9cf8b3dc4d203..bfecd99928a67 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java @@ -42,18 +42,20 @@ /** * Builder for filesystem/filecontext operations of various kinds, * with option support. - * - * + *

+ *
  *   .opt("foofs:option.a", true)
  *   .opt("foofs:option.b", "value")
- *   .opt("barfs:cache", true)
+ *   .opt("fs.s3a.open.option.etag", "9fe4c37c25b")
  *   .must("foofs:cache", true)
  *   .must("barfs:cache-size", 256 * 1024 * 1024)
  *   .build();
- * 
+ * 
+ *

* * Configuration keys declared in an {@code opt()} may be ignored by * a builder which does not recognise them. + *

* * Configuration keys declared in a {@code must()} function set must * be understood by the implementation or a @@ -88,6 +90,9 @@ /** Keep track of the keys for mandatory options. */ private final Set mandatoryKeys = new HashSet<>(); + /** Keep track of the optional keys. */ + private final Set optionalKeys = new HashSet<>(); + /** * Constructor with both optional path and path handle. * Either or both argument may be empty, but it is an error for @@ -163,6 +168,7 @@ public PathHandle getPathHandle() { @Override public B opt(@Nonnull final String key, @Nonnull final String value) { mandatoryKeys.remove(key); + optionalKeys.add(key); options.set(key, value); return getThisBuilder(); } @@ -175,6 +181,7 @@ public B opt(@Nonnull final String key, @Nonnull final String value) { @Override public B opt(@Nonnull final String key, boolean value) { mandatoryKeys.remove(key); + optionalKeys.add(key); options.setBoolean(key, value); return getThisBuilder(); } @@ -187,10 +194,19 @@ public B opt(@Nonnull final String key, boolean value) { @Override public B opt(@Nonnull final String key, int value) { mandatoryKeys.remove(key); + optionalKeys.add(key); options.setInt(key, value); return getThisBuilder(); } + @Override + public B opt(@Nonnull final String key, final long value) { + mandatoryKeys.remove(key); + optionalKeys.add(key); + options.setLong(key, value); + return getThisBuilder(); + } + /** * Set optional float parameter for the Builder. * @@ -199,6 +215,7 @@ public B opt(@Nonnull final String key, int value) { @Override public B opt(@Nonnull final String key, float value) { mandatoryKeys.remove(key); + optionalKeys.add(key); options.setFloat(key, value); return getThisBuilder(); } @@ -211,6 +228,7 @@ public B opt(@Nonnull final String key, float value) { @Override public B opt(@Nonnull final String key, double value) { mandatoryKeys.remove(key); + optionalKeys.add(key); options.setDouble(key, value); return getThisBuilder(); } @@ -223,6 +241,7 @@ public B opt(@Nonnull final String key, double value) { @Override public B opt(@Nonnull final String key, @Nonnull final String... values) { mandatoryKeys.remove(key); + optionalKeys.add(key); options.setStrings(key, values); return getThisBuilder(); } @@ -248,6 +267,7 @@ public B must(@Nonnull final String key, @Nonnull final String value) { @Override public B must(@Nonnull final String key, boolean value) { mandatoryKeys.add(key); + optionalKeys.remove(key); options.setBoolean(key, value); return getThisBuilder(); } @@ -260,10 +280,19 @@ public B must(@Nonnull final String key, boolean value) { @Override public B must(@Nonnull final String key, int value) { mandatoryKeys.add(key); + optionalKeys.remove(key); options.setInt(key, value); return getThisBuilder(); } + @Override + public B must(@Nonnull final String key, final long value) { + mandatoryKeys.add(key); + optionalKeys.remove(key); + options.setLong(key, value); + return getThisBuilder(); + } + /** * Set mandatory float option. * @@ -272,6 +301,7 @@ public B must(@Nonnull final String key, int value) { @Override public B must(@Nonnull final String key, float value) { mandatoryKeys.add(key); + optionalKeys.remove(key); options.setFloat(key, value); return getThisBuilder(); } @@ -284,6 +314,7 @@ public B must(@Nonnull final String key, float value) { @Override public B must(@Nonnull final String key, double value) { mandatoryKeys.add(key); + optionalKeys.remove(key); options.setDouble(key, value); return getThisBuilder(); } @@ -296,6 +327,7 @@ public B must(@Nonnull final String key, double value) { @Override public B must(@Nonnull final String key, @Nonnull final String... values) { mandatoryKeys.add(key); + optionalKeys.remove(key); options.setStrings(key, values); return getThisBuilder(); } @@ -314,6 +346,12 @@ public Configuration getOptions() { public Set getMandatoryKeys() { return Collections.unmodifiableSet(mandatoryKeys); } + /** + * Get all the keys that are set as optional keys. + */ + public Set getOptionalKeys() { + return Collections.unmodifiableSet(optionalKeys); + } /** * Reject a configuration if one or more mandatory keys are diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java index 24a8d49747fe6..04c84d5595196 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java @@ -148,7 +148,7 @@ public FutureDataInputStreamBuilder getThisBuilder() { @Override public FutureDataInputStreamBuilder withFileStatus(FileStatus st) { - this.status = requireNonNull(st, "status"); + this.status = st; return this; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java index 77b4ff52696a3..6e86f8f7afa51 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java @@ -19,6 +19,8 @@ package org.apache.hadoop.fs.impl; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -38,6 +40,9 @@ public class OpenFileParameters { */ private Set mandatoryKeys; + /** The optional keys. */ + private Set optionalKeys; + /** * Options set during the build sequence. */ @@ -61,6 +66,11 @@ public OpenFileParameters withMandatoryKeys(final Set keys) { return this; } + public OpenFileParameters withOptionalKeys(final Set keys) { + this.optionalKeys = requireNonNull(keys); + return this; + } + public OpenFileParameters withOptions(final Configuration opts) { this.options = requireNonNull(opts); return this; @@ -80,6 +90,10 @@ public Set getMandatoryKeys() { return mandatoryKeys; } + public Set getOptionalKeys() { + return optionalKeys; + } + public Configuration getOptions() { return options; } @@ -91,4 +105,48 @@ public int getBufferSize() { public FileStatus getStatus() { return status; } + + + /** + * OpenFile option for seek policies: {@value}. + */ + public static final String FS_OPT_OPENFILE_FADVISE = + "fs.opt.openfile.fadvise"; + + /** + * fadvise policy: {@value}. + */ + public static final String FS_OPT_OPENFILE_FADVISE_NORMAL = "normal"; + + /** + * fadvise policy: {@value}. + */ + public static final String FS_OPT_OPENFILE_FADVISE_SEQUENTIAL = "sequential"; + + /** + * fadvise policy: {@value}. + */ + public static final String FS_OPT_OPENFILE_FADVISE_RANDOM = "random"; + + /** + * fadvise policy: {@value}. + */ + public static final String FS_OPT_OPENFILE_FADVISE_ADAPTIVE = "adaptive"; + + /** + * OpenFile option for seek policies: {@value}. + */ + public static final String FS_OPT_OPENFILE_LENGTH = + "fs.opt.openfile.length"; + + /** + * Set of standard options. + */ + public static final Set STANDARD_OPTIONS = + Stream.of( + FS_OPT_OPENFILE_FADVISE, + FS_OPT_OPENFILE_LENGTH) + .collect(Collectors.toSet()); + + } diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md index eadba174fc1a6..9f80cf04baa05 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md @@ -65,8 +65,29 @@ 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 @@ -77,15 +98,22 @@ 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.opt.openfile.fadvise", "random") + .opt("fs.http.connection.timeout", 30_000L) .withFileStatus(statusFromListing) .build() .get(); ``` +Here the seek policy of `random` has been specified, +with the requirement that the filesystem implementation must understand the option. +And s3a-specific option has been supplied which may be interpreted by any store; +the expectation is that the S3A connector will recognize it. + #### Implementation Notes Checking for supported options must be performed in the `build()` operation. @@ -151,3 +179,121 @@ evaluation, and so will surface when the future is completed: ```java FSDataInputStream in = future.get(); ``` + +## Standard options + +Individual filesystems MAY provide their own set of options for use in the builder. + +There are also some standard options with common names, all of which begin with +`fs.opt.openfile` + +| Name | Type | Description | +|------|------|-------------| +| `fs.opt.openfile.fadvise` | string | seek policy | +| `fs.opt.openfile.length` | long | file length | + +These policies are *not* declared as constants in +public `org.apache.hadoop` classes/interfaces, so as to avoid +applications being unable to link to a specific hadoop release +without that standard option. + +(Implementors: use `org.apache.hadoop.fs.impl.OpenFileParameters`.) + +### Seek Policy Option `fs.opt.openfile.fadvise` + +This is a hint as to what the expected read pattern of an input stream will be. +"sequential" -read a file from start to finish. "Random": client will be +reading data in different parts of the file using a sequence of `seek()/read()` +or via the `PositionedReadable` or `ByteBufferPositionedReadable` APIs. + +Sequential reads may be optimized with prefetching data and/or reading data in larger blocks. +In contrast, random IO performance may be best if little/no prefetching takes place, along +with other possible optimizations. +Queries over columnar formats such as Apach ORC and Apache Parquet +perform such random IO; other data formats are best for sequential reads. +Some applications (e.g. distCp) perform sequential IO even over columnar data. + +What is key is that optimizing reads for seqential reads may impair random performance +-and vice versa. + +Allowing applications to hint what their read policy is allows the filesystem clients +to optimize their interaction with the underlying data stores. + +1. The seek policy is a hint; even if declared as a `must()` option, the filesystem +MAY ignore it. +1. The interpretation/implementation of a policy is a filesystem specific behavior +-and it may change with Hadoop releases and/or specific storage subsystems. +1. If a policy is not recognized, the FileSystem MUST ignore it and revert to +whichever policy is active for that specific FileSystem instance. + +| Policy | Meaning | +|--------|---------| +| `normal` | Default policy for the filesystem | +| `sequential` | Optimize for sequential IO | +| `random` | Optimize for random IO | +| `adaptive` | Adapt seek policy based on read patterns | + + +#### Seek Policy `normal` + +The default policy for the filesystem instance. +Implementation/installation-specific + +#### Seek Policy `sequential` + +Expect sequential reads from the first byte read to the end of the file/until the stream is closed. + + +#### Seek Policy `random` + +Expect `seek()/read()` sequences, or use of `PositionedReadable` or `ByteBufferPositionedReadable` APIs. + +#### Seek Policy `adaptive` + +Try to adapt the seek policy to the read pattern of the application. + +The `normal` policy of the S3A client and the sole policy supported by the `wasb:` client are both +adaptive -they assume sequential IO, but once a backwards seek/positioned read call is made +the stream switches to random IO. + +Other filesystem implementations may wish to adopt similar strategies, and/or extend +the algorithms to detect forward seeks and/or switch from random to sequential IO if +that is considered more efficient. + +Adaptive seek policies have proven effective in the absence of the ability to declare +the seek policy in the `open()` API, so requiring it to be declared, if configurable, +in the cluster/application configuration. However, the switch from sequential to +random seek policies may + +When applications explicitly set the `fs.opt.openfile.fadvise` option, if they +know their read plan, they SHOULD declare which policy is most appropriate. + +_Implementor's Notes_ + +If a policy is unrecognized/unsupported: SHOULD log it and then MUST ignore the option. This is +to support future evolution of policies and implementations + + +_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) + +### File Length hint `fs.opt.openfile.length` + +Declare that a file is expected to be *at least as long as the specified length* + +This can be used by clients to skip querying a remote store for the size of/existence of a file when +opening it, similar to declaring a file status through the `withFileStatus()` option. + +The supplied length MAY be shorter than the actual file. + +This allows worker tasks which only know of of their task's split to declare that as the +length of the file. + +If this option is used by the FileSystem implementation + +* A `length` < 0 MUST be rejected. +* If a file status is supplied along with a value in `fs.opt.openfile.length`; the file status +values take precedence. +* seek/read calls past the length value MAY be rejected, equally: they MAY be accepted. It is simply a hint. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java index a43053180fbf8..effae0382d9af 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java @@ -263,6 +263,7 @@ public void testChainedFailureAwaitFuture() throws Throwable { describe("await Future handles chained failures"); CompletableFuture f = getFileSystem() .openFile(path("testOpenFileUnknownOption")) + .withFileStatus(null) .build(); intercept(RuntimeException.class, "exceptionally", @@ -280,13 +281,34 @@ public void testOpenFileApplyRead() throws Throwable { int len = 4096; createFile(fs, path, true, dataset(len, 0x40, 0x80)); + FileStatus st = fs.getFileStatus(path); CompletableFuture readAllBytes = fs.openFile(path) - .withFileStatus(fs.getFileStatus(path)) + .withFileStatus(st) .build() .thenApply(ContractTestUtils::readStream); assertEquals("Wrong number of bytes read value", len, (long) readAllBytes.get()); + // now reattempt with a new FileStatus and a different path + // other than the final name element + // implementations MUST use path in openFile() call + FileStatus st2 = new FileStatus( + len, false, + st.getReplication(), + st.getBlockSize(), + st.getModificationTime(), + st.getAccessTime(), + st.getPermission(), + st.getOwner(), + st.getGroup(), + new Path("gopher:///localhost:/" + path.getName())); + assertEquals("Wrong number of bytes read value", + len, + (long) fs.openFile(path) + .withFileStatus(st2) + .build() + .thenApply(ContractTestUtils::readStream) + .get()); } @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index 39a41d01c458a..35acf217068ab 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -1611,17 +1611,22 @@ public static int read(InputStream in) { /** * Read a whole stream; downgrades an IOE to a runtime exception. + * Closes the stream afterwards. * @param in input * @return the number of bytes read. * @throws AssertionError on any IOException */ public static long readStream(InputStream in) { - long count = 0; + try { + long count = 0; - while (read(in) >= 0) { - count++; + while (read(in) >= 0) { + count++; + } + return count; + } finally { + IOUtils.cleanupWithLogger(LOG, in); } - return count; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index a1c1d969a8258..80959e8596f19 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -24,6 +24,8 @@ import java.util.concurrent.TimeUnit; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; + /** * All the constants used with the {@link S3AFileSystem}. * @@ -522,7 +524,6 @@ private Constants() { * reading data. * Value: {@value} */ - @InterfaceStability.Unstable public static final String INPUT_FADVISE = "fs.s3a.experimental.input.fadvise"; @@ -530,14 +531,12 @@ private Constants() { * General input. Some seeks, some reads. * Value: {@value} */ - @InterfaceStability.Unstable public static final String INPUT_FADV_NORMAL = "normal"; /** * Optimized for sequential access. * Value: {@value} */ - @InterfaceStability.Unstable public static final String INPUT_FADV_SEQUENTIAL = "sequential"; /** @@ -546,7 +545,6 @@ private Constants() { * more efficient {@code seek()} operations. * Value: {@value} */ - @InterfaceStability.Unstable public static final String INPUT_FADV_RANDOM = "random"; @InterfaceAudience.Private diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index a02947b015262..f046126af9d31 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -110,6 +110,7 @@ import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; +import org.apache.hadoop.fs.s3a.impl.S3AOpenFileHelper; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; import org.apache.hadoop.fs.s3a.impl.RenameOperation; import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder; @@ -169,7 +170,6 @@ import org.apache.hadoop.util.SemaphoredDelegatingExecutor; import org.apache.hadoop.util.concurrent.HadoopExecutors; -import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Invoker.*; @@ -303,6 +303,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private final ListingOperationCallbacks listingOperationCallbacks = new ListingOperationCallbacksImpl(); + /** + * Helper for the openFile() method. + */ + private S3AOpenFileHelper openFileHelper; + /** * Directory policy. */ @@ -480,6 +485,12 @@ public void initialize(URI name, Configuration originalConf) pageSize = intOption(getConf(), BULK_DELETE_PAGE_SIZE, BULK_DELETE_PAGE_SIZE_DEFAULT, 0); listing = new Listing(listingOperationCallbacks, createStoreContext()); + // now the open file logic + openFileHelper = new S3AOpenFileHelper( + inputPolicy, + changeDetectionPolicy, + readAhead, + username); } catch (AmazonClientException e) { // amazon client exception: stop all services then throw the translation stopAllServices(); @@ -1102,59 +1113,37 @@ protected URI canonicalizeUri(URI rawUri) { @Retries.RetryTranslated public FSDataInputStream open(Path f, int bufferSize) throws IOException { - return open(f, Optional.empty(), Optional.empty()); + entryPoint(INVOCATION_OPEN); + return open(qualify(f), openFileHelper.openSimpleFile()); } /** * Opens an FSDataInputStream at the indicated Path. - * if status contains an S3AFileStatus reference, it is used - * and so a HEAD request to the store is avoided. + * if fileInformation contains an FileStatus reference, + * then that is used to avoid a check for the file length/existence * - * @param file the file to open - * @param options configuration options if opened with the builder API. - * @param providedStatus optional file status. + * @param path the file to open + * @param fileInformation information about the file to open * @throws IOException IO failure. */ @Retries.RetryTranslated private FSDataInputStream open( - final Path file, - final Optional options, - final Optional providedStatus) + final Path path, + final S3AOpenFileHelper.OpenFileInformation fileInformation) throws IOException { - entryPoint(INVOCATION_OPEN); - final Path path = qualify(file); - S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path, - providedStatus); - - S3AReadOpContext readContext; - if (options.isPresent()) { - Configuration o = options.get(); - // normal path. Open the file with the chosen seek policy, if different - // from the normal one. - // and readahead. - S3AInputPolicy policy = S3AInputPolicy.getPolicy( - o.get(INPUT_FADVISE, inputPolicy.toString())); - long readAheadRange2 = o.getLong(READAHEAD_RANGE, readAhead); - // TODO support change detection policy from options? - readContext = createReadContext( - fileStatus, - policy, - changeDetectionPolicy, - readAheadRange2); - } else { - readContext = createReadContext( - fileStatus, - inputPolicy, - changeDetectionPolicy, - readAhead); - } + final S3AFileStatus fileStatus = + extractOrFetchSimpleFileStatus(path, fileInformation); + S3AReadOpContext readContext = createReadContext( + fileStatus, + fileInformation.getInputPolicy(), + fileInformation.getChangePolicy(), + fileInformation.getReadAheadRange()); LOG.debug("Opening '{}'", readContext); - return new FSDataInputStream( new S3AInputStream( readContext, - createObjectAttributes(fileStatus), + createObjectAttributes(path, fileStatus), s3)); } @@ -1208,13 +1197,14 @@ private S3ObjectAttributes createObjectAttributes( /** * Create the attributes of an object for subsequent use. + * @param path path -this is used over the file status path. * @param fileStatus file status to build from. * @return attributes to use when building the query. */ private S3ObjectAttributes createObjectAttributes( - final S3AFileStatus fileStatus) { + final Path path, final S3AFileStatus fileStatus) { return createObjectAttributes( - fileStatus.getPath(), + path, fileStatus.getETag(), fileStatus.getVersionId(), fileStatus.getLen()); @@ -1244,7 +1234,7 @@ public FSDataOutputStream create(Path f, FsPermission permission, entryPoint(INVOCATION_CREATE); final Path path = qualify(f); String key = pathToKey(path); - FileStatus status = null; + FileStatus status; try { // get the status or throw an FNFE. // when overwriting, there is no need to look for any existing file, @@ -1555,7 +1545,9 @@ public S3ObjectAttributes createObjectAttributes(final Path path, @Override public S3ObjectAttributes createObjectAttributes( final S3AFileStatus fileStatus) { - return S3AFileSystem.this.createObjectAttributes(fileStatus); + return S3AFileSystem.this.createObjectAttributes( + fileStatus.getPath(), + fileStatus); } @Override @@ -4735,24 +4727,27 @@ protected void setTtlTimeProvider(ITtlTimeProvider ttlTimeProvider) { * @throws IOException IO failure */ @Retries.RetryTranslated - private FSDataInputStream select(final Path source, - final String expression, + private FSDataInputStream select(final Path path, final Configuration options, - final Optional providedStatus) + final S3AOpenFileHelper.OpenFileInformation fileInformation) throws IOException { entryPoint(OBJECT_SELECT_REQUESTS); - requireSelectSupport(source); - final Path path = makeQualified(source); + requireSelectSupport(path); + String expression = fileInformation.getSql(); final S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path, - providedStatus); + fileInformation); // readahead range can be dynamically set - long ra = options.getLong(READAHEAD_RANGE, readAhead); - S3ObjectAttributes objectAttributes = createObjectAttributes(fileStatus); - S3AReadOpContext readContext = createReadContext(fileStatus, inputPolicy, - changeDetectionPolicy, ra); + S3ObjectAttributes objectAttributes = createObjectAttributes( + path, fileStatus); + ChangeDetectionPolicy changePolicy = fileInformation.getChangePolicy(); + S3AReadOpContext readContext = createReadContext( + fileStatus, + fileInformation.getInputPolicy(), + changePolicy, + fileInformation.getReadAheadRange()); - if (changeDetectionPolicy.getSource() != ChangeDetectionPolicy.Source.None + if (changePolicy.getSource() != ChangeDetectionPolicy.Source.None && fileStatus.getETag() != null) { // if there is change detection, and the status includes at least an // etag, @@ -4764,7 +4759,7 @@ private FSDataInputStream select(final Path source, // version in the final read; nor can we check the etag match) ChangeTracker changeTracker = new ChangeTracker(uri.toString(), - changeDetectionPolicy, + changePolicy, readContext.instrumentation.newInputStreamStatistics() .getVersionMismatchCounter(), objectAttributes); @@ -4797,21 +4792,20 @@ private void requireSelectSupport(final Path source) throws } /** - * Extract the status from the optional parameter, querying - * S3Guard/s3 if it is absent. + * Get the status for the file, querying + * S3Guard/s3 if there is none in the fileInformation parameter. * @param path path of the status - * @param optStatus optional status + * @param fileInformation information on the file to open * @return a file status * @throws FileNotFoundException if there is no normal file at that path * @throws IOException IO failure */ private S3AFileStatus extractOrFetchSimpleFileStatus( - final Path path, final Optional optStatus) + final Path path, + final S3AOpenFileHelper.OpenFileInformation fileInformation) throws IOException { - S3AFileStatus fileStatus; - if (optStatus.isPresent()) { - fileStatus = optStatus.get(); - } else { + S3AFileStatus fileStatus = fileInformation.getStatus(); + if (fileStatus == null) { // this looks at S3guard and gets any type of status back, // if it falls back to S3 it does a HEAD only. // therefore: if there is no S3Guard and there is a dir, this @@ -4844,54 +4838,19 @@ public CompletableFuture openFileWithOptions( final Path rawPath, final OpenFileParameters parameters) throws IOException { final Path path = qualify(rawPath); - Configuration options = parameters.getOptions(); - Set mandatoryKeys = parameters.getMandatoryKeys(); - String sql = options.get(SelectConstants.SELECT_SQL, null); - boolean isSelect = sql != null; - // choice of keys depends on open type - if (isSelect) { - rejectUnknownMandatoryKeys( - mandatoryKeys, - InternalSelectConstants.SELECT_OPTIONS, - "for " + path + " in S3 Select operation"); - } else { - rejectUnknownMandatoryKeys( - mandatoryKeys, - InternalConstants.STANDARD_OPENFILE_KEYS, - "for " + path + " in non-select file I/O"); - } - FileStatus providedStatus = parameters.getStatus(); - S3AFileStatus fileStatus; - if (providedStatus != null) { - Preconditions.checkArgument(path.equals(providedStatus.getPath()), - "FileStatus parameter is not for the path %s: %s", - path, providedStatus); - if (providedStatus instanceof S3AFileStatus) { - // can use this status to skip our own probes, - // including etag and version. - LOG.debug("File was opened with a supplied S3AFileStatus;" - + " skipping getFileStatus call in open() operation: {}", - providedStatus); - fileStatus = (S3AFileStatus) providedStatus; - } else if (providedStatus instanceof S3ALocatedFileStatus) { - LOG.debug("File was opened with a supplied S3ALocatedFileStatus;" - + " skipping getFileStatus call in open() operation: {}", - providedStatus); - fileStatus = ((S3ALocatedFileStatus) providedStatus).toS3AFileStatus(); - } else { - LOG.debug("Ignoring file status {}", providedStatus); - fileStatus = null; - } - } else { - fileStatus = null; - } - Optional ost = Optional.ofNullable(fileStatus); + S3AOpenFileHelper.OpenFileInformation fileInformation = + openFileHelper.prepareToOpenFile( + path, + parameters, + getDefaultBlockSize(path)); + boolean isSelect = fileInformation.isSql(); CompletableFuture result = new CompletableFuture<>(); + Configuration options = parameters.getOptions(); if (!isSelect) { // normal path. unboundedThreadPool.submit(() -> LambdaUtils.eval(result, - () -> open(path, Optional.of(options), ost))); + () -> open(path, fileInformation))); } else { // it is a select statement. // fail fast if the operation is not available @@ -4899,7 +4858,7 @@ public CompletableFuture openFileWithOptions( // submit the query unboundedThreadPool.submit(() -> LambdaUtils.eval(result, - () -> select(path, sql, options, ost))); + () -> select(path, options, fileInformation))); } return result; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java index c018410e11767..bc726fea4eb95 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java @@ -25,6 +25,7 @@ import java.util.Locale; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE_ADAPTIVE; import static org.apache.hadoop.fs.s3a.Constants.*; /** @@ -62,6 +63,7 @@ public static S3AInputPolicy getPolicy(String name) { String trimmed = name.trim().toLowerCase(Locale.ENGLISH); switch (trimmed) { case INPUT_FADV_NORMAL: + case FS_OPT_OPENFILE_FADVISE_ADAPTIVE: return Normal; case INPUT_FADV_RANDOM: return Random; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index 542fe34e96c79..d6a281c8ebaa8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -142,7 +142,7 @@ public S3AInputStream(S3AReadOpContext ctx, this.context = ctx; this.bucket = s3Attributes.getBucket(); this.key = s3Attributes.getKey(); - this.pathStr = ctx.dstFileStatus.getPath().toString(); + this.pathStr = s3Attributes.getPath().toString(); this.contentLength = l; this.client = client; this.uri = "s3a://" + this.bucket + "/" + this.key; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 9ecc1a93c60e2..ee09c9cef5c14 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -18,16 +18,18 @@ package org.apache.hadoop.fs.s3a.impl; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.s3a.Constants; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; + /** * Internal constants private only to the S3A codebase. * Please don't refer to these outside of this module & its tests. @@ -79,11 +81,14 @@ private InternalConstants() { * used becomes that of the select operation. */ @InterfaceStability.Unstable - public static final Set STANDARD_OPENFILE_KEYS = - Collections.unmodifiableSet( - new HashSet<>( - Arrays.asList(Constants.INPUT_FADVISE, - Constants.READAHEAD_RANGE))); + public static final Set S3A_OPENFILE_KEYS = + Stream.of( + FS_OPT_OPENFILE_FADVISE, + FS_OPT_OPENFILE_LENGTH, + Constants.INPUT_FADVISE, + Constants.READAHEAD_RANGE) + .collect(Collectors.toSet()); + /** 404 error code. */ public static final int SC_404 = 404; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileHelper.java new file mode 100644 index 0000000000000..bb467613e29ad --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileHelper.java @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.impl.OpenFileParameters; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AInputPolicy; +import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; +import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; +import org.apache.hadoop.fs.s3a.select.SelectConstants; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; +import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; + +/** + * Helper class for openFile() logic, especially processing file status + * args and length/etag/versionID. + *

+ * This got complex enough it merited removal from S3AFS -which + * also permits unit testing. + */ + public class S3AOpenFileHelper { + + private static final Logger LOG = + LoggerFactory.getLogger(S3AOpenFileHelper.class); + + private final S3AInputPolicy inputPolicy; + + private final ChangeDetectionPolicy changePolicy; + + private final long readAheadRange; + + private final String username; + + /** + * Simple file information is created in advance as it is always + * the same. + */ + private final OpenFileInformation simpleFileInformation; + + + /** + * Instantiate with the default options from the filesystem. + * @param inputPolicy input policy + * @param changePolicy change detection policy + * @param readAheadRange read ahead range + * @param username username + */ + public S3AOpenFileHelper( + final S3AInputPolicy inputPolicy, + final ChangeDetectionPolicy changePolicy, + final long readAheadRange, + final String username) { + this.inputPolicy = inputPolicy; + this.changePolicy = changePolicy; + this.readAheadRange = readAheadRange; + this.username = username; + + simpleFileInformation = new OpenFileInformation(false, null, null, + inputPolicy, changePolicy, readAheadRange); + } + + + /** + * The information on a file needed to open it. + */ + public static final class OpenFileInformation { + + /** Is this SQL? */ + private final boolean isSql; + + /** File status; may be null. */ + private final S3AFileStatus status; + + /** SQL string if this is a SQL select file*/ + private final String sql; + + /** Active input policy. */ + private final S3AInputPolicy inputPolicy; + + /** Change detection policy. */ + private final ChangeDetectionPolicy changePolicy; + + /** read ahead range. */ + private final long readAheadRange; + + private OpenFileInformation(final boolean isSql, + final S3AFileStatus status, + final String sql, + final S3AInputPolicy inputPolicy, + final ChangeDetectionPolicy changePolicy, + final long readAheadRange) { + this.isSql = isSql; + this.status = status; + this.sql = sql; + this.inputPolicy = inputPolicy; + this.changePolicy = changePolicy; + this.readAheadRange = readAheadRange; + } + + public boolean isSql() { + return isSql; + } + + public S3AFileStatus getStatus() { + return status; + } + + public String getSql() { + return sql; + } + + public S3AInputPolicy getInputPolicy() { + return inputPolicy; + } + + public ChangeDetectionPolicy getChangePolicy() { + return changePolicy; + } + + public long getReadAheadRange() { + return readAheadRange; + } + + @Override + public String toString() { + return "OpenFileInformation{" + + "isSql=" + isSql + + ", status=" + status + + ", sql='" + sql + '\'' + + ", inputPolicy=" + inputPolicy + + ", changePolicy=" + changePolicy + + ", readAheadRange=" + readAheadRange + + '}'; + } + } + + /** + * Prepare to open a file from the openFile parameters. + * @param path path to the file + * @param parameters open file parameters from the builder. + * @param blockSize for fileStatus + * @return open file options + * @throws IOException failure to resolve the link. + * @throws IllegalArgumentException unknown mandatory key + */ + public OpenFileInformation prepareToOpenFile( + final Path path, + final OpenFileParameters parameters, + final long blockSize) throws IOException { + Configuration options = parameters.getOptions(); + Set mandatoryKeys = parameters.getMandatoryKeys(); + String sql = options.get(SelectConstants.SELECT_SQL, null); + boolean isSelect = sql != null; + // choice of keys depends on open type + if (isSelect) { + // S3 Select call adds a large set of supported mandatory keys + rejectUnknownMandatoryKeys( + mandatoryKeys, + InternalSelectConstants.SELECT_OPTIONS, + "for " + path + " in S3 Select operation"); + } else { + rejectUnknownMandatoryKeys( + mandatoryKeys, + InternalConstants.S3A_OPENFILE_KEYS, + "for " + path + " in non-select file I/O"); + } + FileStatus providedStatus = parameters.getStatus(); + S3AFileStatus fileStatus; + if (providedStatus != null) { + // there's a file status + + // make sure the file name matches -the rest of the path + // MUST NOT be checked. + Path providedStatusPath = providedStatus.getPath(); + checkArgument(path.getName().equals(providedStatusPath.getName()), + "Filename mismatch between file being opened %s and" + + " supplied filestatus %s", + path, providedStatusPath); + + // make sure the status references a file + if (providedStatus.isDirectory()) { + throw new FileNotFoundException( + "Supplied status references a directory " + providedStatus); + } + // build up the values + long len = providedStatus.getLen(); + long modTime = providedStatus.getModificationTime(); + String versionId; + String eTag; + // can use this status to skip our own probes, + LOG.debug("File was opened with a supplied FileStatus;" + + " skipping getFileStatus call in open() operation: {}", + + providedStatus); + if (providedStatus instanceof S3AFileStatus) { + S3AFileStatus st = (S3AFileStatus) providedStatus; + versionId = st.getVersionId(); + eTag = st.getETag(); + } else if (providedStatus instanceof S3ALocatedFileStatus) { + // S3ALocatedFileStatus instance may supply etag and version. + S3ALocatedFileStatus st + = (S3ALocatedFileStatus) providedStatus; + versionId = st.getVersionId(); + eTag = st.getETag(); + } else { + // it is another type. + // build a status struct without etag or version. + LOG.debug("Converting file status {}", providedStatus); + versionId = null; + eTag = null; + } + // Construct a new file status with the real path of the file. + fileStatus = new S3AFileStatus( + len, + modTime, + path, + blockSize, + username, + eTag, + versionId); + } else if (options.get(FS_OPT_OPENFILE_LENGTH) != null) { + // build a minimal S3A FileStatus From the input length alone. + // this is all we actually need. + long length = options.getLong(FS_OPT_OPENFILE_LENGTH, 0); + LOG.debug("Fixing length of file to read {} as {}", path, length); + fileStatus = new S3AFileStatus( + length, + 0, + path, + blockSize, + username, + null, null); + } else { + // neither a file status nor a file length + fileStatus = null; + } + // seek policy from default, s3a opt or standard option + String policy1 = options.get(INPUT_FADVISE, inputPolicy.toString()); + String policy2 = options.get(FS_OPT_OPENFILE_FADVISE, policy1); + S3AInputPolicy policy = S3AInputPolicy.getPolicy(policy2); + + // readahead range + long ra = options.getLong(READAHEAD_RANGE, readAheadRange); + return new OpenFileInformation(isSelect, fileStatus, sql, policy, + changePolicy, ra); + } + + /** + * Open a simple file by builing the base + * @return the parameters needed to open a file through open(). + */ + public OpenFileInformation openSimpleFile() { + return simpleFileInformation; + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java index 4f387d8ccfa75..fbf5226afb82f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java @@ -71,7 +71,7 @@ private InternalSelectConstants() { CSV_OUTPUT_QUOTE_FIELDS, CSV_OUTPUT_RECORD_DELIMITER )); - options.addAll(InternalConstants.STANDARD_OPENFILE_KEYS); + options.addAll(InternalConstants.S3A_OPENFILE_KEYS); SELECT_OPTIONS = Collections.unmodifiableSet(options); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java index d78273b14710a..e0d9d4d9a06b1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java @@ -18,11 +18,22 @@ package org.apache.hadoop.fs.contract.s3a; +import java.io.FileNotFoundException; + +import org.junit.Test; + import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.AbstractContractOpenTest; import org.apache.hadoop.fs.contract.AbstractFSContract; +import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; +import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * S3A contract tests opening files. @@ -54,4 +65,59 @@ protected AbstractFSContract createContract(Configuration conf) { protected boolean areZeroByteFilesEncrypted() { return true; } + + @Test + public void testOpenFileApplyReadBadName() throws Throwable { + describe("use the apply sequence to read a whole file"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + touch(fs, path); + FileStatus st = fs.getFileStatus(path); + // The final element of the path is different, so + // openFile must fail + FileStatus st2 = new FileStatus( + 0, false, + st.getReplication(), + st.getBlockSize(), + st.getModificationTime(), + st.getAccessTime(), + st.getPermission(), + st.getOwner(), + st.getGroup(), + new Path("gopher:///localhost/something.txt")); + intercept(IllegalArgumentException.class, () -> + fs.openFile(path) + .withFileStatus(st2) + .build()); + } + + /** + * Pass in a directory reference and expect the openFile call + * to fail. + */ + @Test + public void testOpenFileDirectory() throws Throwable { + describe("Change the status to a directory"); + Path path = methodPath(); + FileSystem fs = getFileSystem(); + int len = 4096; + createFile(fs, path, true, + dataset(len, 0x40, 0x80)); + FileStatus st = fs.getFileStatus(path); + FileStatus st2 = new FileStatus( + len, true, + st.getReplication(), + st.getBlockSize(), + st.getModificationTime(), + st.getAccessTime(), + st.getPermission(), + st.getOwner(), + st.getGroup(), + path); + intercept(FileNotFoundException.class, () -> + fs.openFile(path) + .withFileStatus(st2) + .build()); + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java index ca8e49cc33be2..f9e0a2b89d635 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java @@ -18,7 +18,9 @@ package org.apache.hadoop.fs.s3a; - +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -33,15 +35,22 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.EOFException; import java.io.File; import java.io.FileNotFoundException; import java.net.URI; import java.util.Arrays; import java.util.Collection; import java.util.EnumSet; - +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.performance.OperationCost.*; @@ -505,5 +514,98 @@ public void testCostOfGlobStatusNoSymlinkResolution() throws Throwable { () -> fs.globStatus(basePath.suffix("/*"))); } + /** + * Test when openFile() performs GET requests when file status + * and length options are passed down. + * Note that the input streams only update the FS statistics + * in close(), so metrics cannot be verified until all operations + * on a stream are complete. + * This is slightly less than ideal. + */ + @Test + public void testOpenFileCost() throws Throwable { + describe("Test cost of openFile with/without status; raw only"); + S3AFileSystem fs = getFileSystem(); + assume("Unguarded FS only", !fs.hasMetadataStore()); + Path testFile = path("testOpenFileCost"); + + writeTextFile(fs, testFile, "openfile", true); + FileStatus st = fs.getFileStatus(testFile); + + // now read that file back in using the openFile call. + // with a new FileStatus and a different path. + // this verifies that any FileStatus class/subclass is used + // as a source of the file length. + long len = st.getLen(); + FileStatus st2 = new FileStatus( + len, false, + st.getReplication(), + st.getBlockSize(), + st.getModificationTime(), + st.getAccessTime(), + st.getPermission(), + st.getOwner(), + st.getGroup(), + new Path("gopher:///localhost/" + testFile.getName())); + resetMetricDiffs(); + // measure GET requests issued. + MetricDiff getRequests = new MetricDiff(fs, STREAM_OPENED); + + final CompletableFuture future = + fs.openFile(testFile) + .withFileStatus(st) + .build(); + FSDataInputStream in = future.get(); + verifyOperationCount(0, 0); + getRequests.assertDiffEquals(0); + + long readLen = readStream(in); + assertEquals("bytes read from file", len, readLen); + verifyOperationCount(0, 0); + getRequests.assertDiffEquals(1); + getRequests.reset(); + resetMetricDiffs(); + + // do a second read with the length declared as short. + // we now expect the bytes read to be shorter. + MetricDiff bytesDiscarded = + new MetricDiff(fs, STREAM_CLOSE_BYTES_READ); + int offset = 2; + long shortLen = len - offset; + CompletableFuture f2 = fs.openFile(testFile) + .must(FS_OPT_OPENFILE_FADVISE, FS_OPT_OPENFILE_FADVISE_SEQUENTIAL) + .opt(FS_OPT_OPENFILE_LENGTH, shortLen) + .build(); + verifyOperationCount(0, 0); + getRequests.assertDiffEquals(0); + FSDataInputStream in2 = f2.get(); + long r2 = readStream(in2); + verifyOperationCount(0, 0); + getRequests.assertDiffEquals(1); + assertEquals("bytes read from file", shortLen, r2); + // the read has been ranged + bytesDiscarded.assertDiffEquals(0); + + // final read past EOF + getRequests.reset(); + + long longLen = len + 10; + FSDataInputStream in3 = fs.openFile(testFile) + .must(FS_OPT_OPENFILE_FADVISE, FS_OPT_OPENFILE_FADVISE_SEQUENTIAL) + .must(FS_OPT_OPENFILE_LENGTH, longLen) + .build() + .get(); + byte[] out = new byte[(int) longLen]; + intercept(EOFException.class, + () -> in3.readFully(0, out)); + in3.seek(longLen - 1); + assertEquals("read past real EOF on " + in3, + -1, in3.read()); + in3.close(); + // two GET calls were made, one for readFully, + // the second on the read() past the EOF + // the operation has got as far as S3 + getRequests.assertDiffEquals(2); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java new file mode 100644 index 0000000000000..f3ac569843454 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import java.io.FileNotFoundException; + +import org.assertj.core.api.Assertions; +import org.assertj.core.api.ObjectAssert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.impl.OpenFileParameters; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AInputPolicy; +import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; +import org.apache.hadoop.test.HadoopTestBase; + +import static java.util.Collections.singleton; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE_ADAPTIVE; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE_RANDOM; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; +import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Unit tests for {@link S3AOpenFileHelper}. + */ +public class TestOpenFileHelper extends HadoopTestBase { + + private static final ChangeDetectionPolicy CHANGE_POLICY + = ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.Server, + ChangeDetectionPolicy.Source.None, false); + + private static final long READ_AHEAD_RANGE = 16; + + private static final String USERNAME = "hadoop"; + + public static final S3AInputPolicy INPUT_POLICY = S3AInputPolicy.Sequential; + + public static final String TESTFILE = "s3a://bucket/name"; + + private static final Path TESTPATH = new Path(TESTFILE); + + private S3AOpenFileHelper helper; + + @Before + public void setup() { + helper = new S3AOpenFileHelper(INPUT_POLICY, + CHANGE_POLICY, READ_AHEAD_RANGE, USERNAME); + } + + @Test + public void testSimpleFile() throws Throwable { + ObjectAssert + asst = assertFI(helper.openSimpleFile()); + + asst.extracting(f -> f.getChangePolicy()) + .isEqualTo(CHANGE_POLICY); + asst.extracting(f -> f.getInputPolicy()) + .isEqualTo(INPUT_POLICY); + asst.extracting(f -> f.getReadAheadRange()) + .isEqualTo(READ_AHEAD_RANGE); + } + + private ObjectAssert assertFI( + final S3AOpenFileHelper.OpenFileInformation fi) { + return Assertions.assertThat(fi) + .describedAs("File Information %s", fi); + } + + @Test + public void testUnknownMandatory() throws Throwable { + + String key = "unknown"; + intercept(IllegalArgumentException.class, key, () -> + helper.prepareToOpenFile(TESTPATH, + params(key, "undefined"), + 0)); + } + + @Test + public void testSeekPolicy() throws Throwable { + + // ask for random IO + ObjectAssert asst = + assertFI(helper.prepareToOpenFile(TESTPATH, + params(INPUT_FADVISE, FS_OPT_OPENFILE_FADVISE_RANDOM), + 0)); + + // is picked up + asst.extracting(f -> f.getInputPolicy()) + .isEqualTo(S3AInputPolicy.Random); + // and as neither status nor length was set: no file status + asst.extracting(f -> f.getStatus()) + .isNull(); + } + + @Test + public void testSeekPolicyAdaptive() throws Throwable { + + // when caller asks for adaptive, they get "normal" + ObjectAssert asst = + assertFI(helper.prepareToOpenFile(TESTPATH, + params(INPUT_FADVISE, FS_OPT_OPENFILE_FADVISE_ADAPTIVE), + 0)); + + asst.extracting(f -> f.getInputPolicy()) + .isEqualTo(S3AInputPolicy.Normal); + } + + @Test + public void testUnknownSeekPolicy() throws Throwable { + + // fall back to the normal seek policy. + ObjectAssert asst = + assertFI(helper.prepareToOpenFile(TESTPATH, + params(INPUT_FADVISE, "undefined"), + 0)); + + asst.extracting(f -> f.getInputPolicy()) + .isEqualTo(S3AInputPolicy.Normal); + } + + @Test + public void testReadahead() throws Throwable { + + // readahead range option + ObjectAssert asst = + assertFI(helper.prepareToOpenFile(TESTPATH, + params(READAHEAD_RANGE, "4096"), + 0)); + + asst.extracting(f -> f.getReadAheadRange()) + .isEqualTo(4096L); + } + + @Test + public void testStatusWithValidFilename() throws Throwable { + Path p = new Path("file:///tmp/" + TESTPATH.getName()); + ObjectAssert asst = + assertFI(helper.prepareToOpenFile(TESTPATH, + params(FS_OPT_OPENFILE_LENGTH, "32") + .withStatus(status(p, 4096)), + 0)); + asst.extracting(f -> f.getStatus().getVersionId()) + .isEqualTo("version"); + asst.extracting(f -> f.getStatus().getETag()) + .isEqualTo("etag"); + asst.extracting(f -> f.getStatus().getLen()) + .isEqualTo(4096L); + } + + @Test + public void testLocatedStatus() throws Throwable { + Path p = new Path("file:///tmp/" + TESTPATH.getName()); + ObjectAssert asst = + assertFI(helper.prepareToOpenFile(TESTPATH, + params(FS_OPT_OPENFILE_LENGTH, "32") + .withStatus( + new S3ALocatedFileStatus( + status(p, 4096), null)), + 0)); + asst.extracting(f -> f.getStatus().getVersionId()) + .isEqualTo("version"); + asst.extracting(f -> f.getStatus().getETag()) + .isEqualTo("etag"); + asst.extracting(f -> f.getStatus().getLen()) + .isEqualTo(4096L); + } + + /** + * Callers cannot supply a directory status when opening a file. + */ + @Test + public void testDirectoryStatus() throws Throwable { + intercept(FileNotFoundException.class, TESTFILE, () -> + helper.prepareToOpenFile(TESTPATH, + params(INPUT_FADVISE, "normal") + .withStatus(new S3AFileStatus(true, TESTPATH, USERNAME)), + 0)); + } + + /** + * File name must match the path argument to openFile(). + */ + @Test + public void testStatusWithInconsistentFilename() throws Throwable { + intercept(IllegalArgumentException.class, TESTFILE, () -> + helper.prepareToOpenFile(TESTPATH, + params(INPUT_FADVISE, "normal") + .withStatus(new S3AFileStatus(true, + new Path(TESTFILE + "-"), USERNAME)), + 0)); + } + + /** + * If a file length option is set, create a file status + */ + @Test + public void testFileLength() throws Throwable { + ObjectAssert asst = + assertFI(helper.prepareToOpenFile(TESTPATH, + params(FS_OPT_OPENFILE_LENGTH, "8192") + .withStatus(null), + 0)); + asst.extracting(f -> f.getStatus()) + .isNotNull(); + asst.extracting(f -> f.getStatus().getPath()) + .isEqualTo(TESTPATH); + asst.extracting(f -> f.getStatus().getLen()) + .isEqualTo(8192L); + } + + + private S3AFileStatus status(final Path p, final int length) { + return new S3AFileStatus(length, 0, + p, 0, "", "etag", "version"); + } + + private OpenFileParameters params(final String key, final String val) { + return new OpenFileParameters() + .withMandatoryKeys(singleton(key)) + .withOptions(conf(key, val)); + } + + private Configuration conf(String key, Object val) { + Configuration c = new Configuration(false); + c.set(key, val.toString()); + return c; + } +} From 89a16420fb91c934de3b0d0b55df78ae9fbac285 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 3 Sep 2020 19:29:54 +0100 Subject: [PATCH 02/10] HADOOP-16202: resync and move to new operation cost testing Change-Id: I91de02318af2bb042c7d4dfa65d418c363edee50 --- .../fs/s3a/ITestS3AFileOperationCost.java | 106 +--------- .../s3a/performance/AbstractS3ACostTest.java | 5 +- .../fs/s3a/performance/ITestS3AOpenCost.java | 184 ++++++++++++++++++ 3 files changed, 190 insertions(+), 105 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java index f9e0a2b89d635..ca8e49cc33be2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java @@ -18,9 +18,7 @@ package org.apache.hadoop.fs.s3a; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; + import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -35,22 +33,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.EOFException; import java.io.File; import java.io.FileNotFoundException; import java.net.URI; import java.util.Arrays; import java.util.Collection; import java.util.EnumSet; -import java.util.UUID; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; + import static org.apache.hadoop.fs.contract.ContractTestUtils.*; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE_SEQUENTIAL; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; -import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.performance.OperationCost.*; @@ -514,98 +505,5 @@ public void testCostOfGlobStatusNoSymlinkResolution() throws Throwable { () -> fs.globStatus(basePath.suffix("/*"))); } - /** - * Test when openFile() performs GET requests when file status - * and length options are passed down. - * Note that the input streams only update the FS statistics - * in close(), so metrics cannot be verified until all operations - * on a stream are complete. - * This is slightly less than ideal. - */ - @Test - public void testOpenFileCost() throws Throwable { - describe("Test cost of openFile with/without status; raw only"); - S3AFileSystem fs = getFileSystem(); - assume("Unguarded FS only", !fs.hasMetadataStore()); - Path testFile = path("testOpenFileCost"); - - writeTextFile(fs, testFile, "openfile", true); - FileStatus st = fs.getFileStatus(testFile); - - // now read that file back in using the openFile call. - // with a new FileStatus and a different path. - // this verifies that any FileStatus class/subclass is used - // as a source of the file length. - long len = st.getLen(); - FileStatus st2 = new FileStatus( - len, false, - st.getReplication(), - st.getBlockSize(), - st.getModificationTime(), - st.getAccessTime(), - st.getPermission(), - st.getOwner(), - st.getGroup(), - new Path("gopher:///localhost/" + testFile.getName())); - resetMetricDiffs(); - // measure GET requests issued. - MetricDiff getRequests = new MetricDiff(fs, STREAM_OPENED); - - final CompletableFuture future = - fs.openFile(testFile) - .withFileStatus(st) - .build(); - FSDataInputStream in = future.get(); - verifyOperationCount(0, 0); - getRequests.assertDiffEquals(0); - - long readLen = readStream(in); - assertEquals("bytes read from file", len, readLen); - verifyOperationCount(0, 0); - getRequests.assertDiffEquals(1); - getRequests.reset(); - resetMetricDiffs(); - - // do a second read with the length declared as short. - // we now expect the bytes read to be shorter. - MetricDiff bytesDiscarded = - new MetricDiff(fs, STREAM_CLOSE_BYTES_READ); - int offset = 2; - long shortLen = len - offset; - CompletableFuture f2 = fs.openFile(testFile) - .must(FS_OPT_OPENFILE_FADVISE, FS_OPT_OPENFILE_FADVISE_SEQUENTIAL) - .opt(FS_OPT_OPENFILE_LENGTH, shortLen) - .build(); - verifyOperationCount(0, 0); - getRequests.assertDiffEquals(0); - FSDataInputStream in2 = f2.get(); - long r2 = readStream(in2); - verifyOperationCount(0, 0); - getRequests.assertDiffEquals(1); - assertEquals("bytes read from file", shortLen, r2); - // the read has been ranged - bytesDiscarded.assertDiffEquals(0); - - // final read past EOF - getRequests.reset(); - - long longLen = len + 10; - FSDataInputStream in3 = fs.openFile(testFile) - .must(FS_OPT_OPENFILE_FADVISE, FS_OPT_OPENFILE_FADVISE_SEQUENTIAL) - .must(FS_OPT_OPENFILE_LENGTH, longLen) - .build() - .get(); - byte[] out = new byte[(int) longLen]; - intercept(EOFException.class, - () -> in3.readFully(0, out)); - in3.seek(longLen - 1); - assertEquals("read past real EOF on " + in3, - -1, in3.read()); - in3.close(); - // two GET calls were made, one for readFully, - // the second on the read() past the EOF - // the operation has got as far as S3 - getRequests.assertDiffEquals(2); - } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index 2b2fb7c9caca0..6d636516cd046 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -164,7 +164,10 @@ public void setup() throws Exception { OBJECT_LIST_REQUESTS, OBJECT_METADATA_REQUESTS, OBJECT_PUT_BYTES, - OBJECT_PUT_REQUESTS) + OBJECT_PUT_REQUESTS, + STREAM_OPENED, + STREAM_CLOSE_BYTES_READ, + STREAM_SEEK_BYTES_SKIPPED) .build(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java new file mode 100644 index 0000000000000..51c34a81ee903 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.hadoop.fs.s3a.performance; + + +import java.io.EOFException; +import java.util.Arrays; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3ATestUtils; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.readStream; +import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.Statistic.STREAM_CLOSE_BYTES_READ; +import static org.apache.hadoop.fs.s3a.Statistic.STREAM_OPENED; +import static org.apache.hadoop.fs.s3a.Statistic.STREAM_SEEK_BYTES_SKIPPED; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.NO_IO; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Cost of openFile(). + */ +@RunWith(Parameterized.class) +public class ITestS3AOpenCost extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestS3AOpenCost.class); + + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"raw-keep-markers", false, true, false}, + }); + } + + public ITestS3AOpenCost(final String name, + final boolean s3guard, + final boolean keepMarkers, + final boolean authoritative) { + super(s3guard, keepMarkers, authoritative); + } + + /** + * Test when openFile() performs GET requests when file status + * and length options are passed down. + * Note that the input streams only update the FS statistics + * in close(), so metrics cannot be verified until all operations + * on a stream are complete. + * This is slightly less than ideal. + */ + @Test + public void testOpenFileCost() throws Throwable { + describe("Test cost of openFile with/without status; raw only"); + S3AFileSystem fs = getFileSystem(); + Path testFile = path("testOpenFileCost"); + + writeTextFile(fs, testFile, "openfile", true); + FileStatus st = fs.getFileStatus(testFile); + + // now read that file back in using the openFile call. + // with a new FileStatus and a different path. + // this verifies that any FileStatus class/subclass is used + // as a source of the file length. + long len = st.getLen(); + FileStatus st2 = new FileStatus( + len, false, + st.getReplication(), + st.getBlockSize(), + st.getModificationTime(), + st.getAccessTime(), + st.getPermission(), + st.getOwner(), + st.getGroup(), + new Path("gopher:///localhost/" + testFile.getName())); + + // no IO + FSDataInputStream in = verifyMetrics(() -> + fs.openFile(testFile) + .withFileStatus(st) + .build() + .get(), + always(NO_IO), + with(STREAM_OPENED, 0)); + + long readLen = verifyMetrics(() -> + readStream(in), + always(NO_IO), + with(STREAM_OPENED, 1)); + + + assertEquals("bytes read from file", len, readLen); + + // do a second read with the length declared as short. + // we now expect the bytes read to be shorter. + S3ATestUtils.MetricDiff bytesDiscarded = + new S3ATestUtils.MetricDiff(fs, STREAM_CLOSE_BYTES_READ); + int offset = 2; + long shortLen = len - offset; + CompletableFuture f2 = fs.openFile(testFile) + .must(FS_OPT_OPENFILE_FADVISE, FS_OPT_OPENFILE_FADVISE_SEQUENTIAL) + .opt(FS_OPT_OPENFILE_LENGTH, shortLen) + .build(); + FSDataInputStream in2 = verifyMetrics(() -> + fs.openFile(testFile) + .must(FS_OPT_OPENFILE_FADVISE, + FS_OPT_OPENFILE_FADVISE_SEQUENTIAL) + .opt(FS_OPT_OPENFILE_LENGTH, shortLen) + .build() + .get(), + always(NO_IO), + with(STREAM_OPENED, 0)); + + long r2 = verifyMetrics(() -> + readStream(in2), + always(NO_IO), + with(STREAM_OPENED, 1), + with(STREAM_CLOSE_BYTES_READ, 0), + with(STREAM_SEEK_BYTES_SKIPPED, 0)); + + assertEquals("bytes read from file", shortLen, r2); + // the read has been ranged + bytesDiscarded.assertDiffEquals(0); + + long longLen = len + 10; + FSDataInputStream in3 = verifyMetrics(() -> + fs.openFile(testFile) + .must(FS_OPT_OPENFILE_FADVISE, FS_OPT_OPENFILE_FADVISE_SEQUENTIAL) + .must(FS_OPT_OPENFILE_LENGTH, longLen) + .build() + .get(), + always(NO_IO)); + + // shows that irrespective of the declared length, you can read past it. + verifyMetrics(() -> { + byte[] out = new byte[(int) longLen]; + intercept(EOFException.class, + () -> in3.readFully(0, out)); + in3.seek(longLen - 1); + assertEquals("read past real EOF on " + in3, + -1, in3.read()); + in3.close(); + return null; + }, + // two GET calls were made, one for readFully, + // the second on the read() past the EOF + // the operation has got as far as S3 + with(STREAM_OPENED, 2)); + + } +} From 8d03de30873bd2740f5d52f7bdcea6d750f8373f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 4 Sep 2020 14:56:40 +0100 Subject: [PATCH 03/10] 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 --- .../java/org/apache/hadoop/fs/FileSystem.java | 2 +- .../fs/FutureDataInputStreamBuilder.java | 12 +- .../org/apache/hadoop/fs/OpenFileOptions.java | 92 +++++++ .../FutureDataInputStreamBuilderImpl.java | 11 +- .../hadoop/fs/impl/OpenFileParameters.java | 43 ---- .../site/markdown/filesystem/filesystem.md | 90 +------ .../src/site/markdown/filesystem/index.md | 2 + .../src/site/markdown/filesystem/openfile.md | 241 ++++++++++++++++++ .../fs/contract/AbstractContractOpenTest.java | 42 ++- .../org/apache/hadoop/fs/s3a/Constants.java | 2 - .../apache/hadoop/fs/s3a/S3AFileSystem.java | 16 +- .../apache/hadoop/fs/s3a/S3AInputPolicy.java | 4 +- .../fs/s3a/impl/AbstractStoreOperation.java | 6 +- .../hadoop/fs/s3a/impl/InternalConstants.java | 8 +- ...eHelper.java => S3AOpenFileOperation.java} | 17 +- .../fs/s3a/ITestS3ARemoteFileChanged.java | 4 +- .../fs/s3a/impl/TestOpenFileHelper.java | 42 +-- .../fs/s3a/performance/ITestS3AOpenCost.java | 22 +- 18 files changed, 447 insertions(+), 209 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/OpenFileOptions.java create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/{S3AOpenFileHelper.java => S3AOpenFileOperation.java} (94%) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index e0f566d94b7c6..940f7dbeb3d6d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -4655,7 +4655,7 @@ protected CompletableFuture openFileWithOptions( final OpenFileParameters parameters) throws IOException { AbstractFSBuilderImpl.rejectUnknownMandatoryKeys( parameters.getMandatoryKeys(), - OpenFileParameters.STANDARD_OPTIONS, ""); + OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS, ""); CompletableFuture result = new CompletableFuture<>(); try { result.complete(open(pathHandle, parameters.getBufferSize())); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java index 27a522e593001..e2601a0b5ae2c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.fs; +import javax.annotation.Nullable; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -27,14 +28,14 @@ * Builder for input streams and subclasses whose return value is * actually a completable future: this allows for better asynchronous * operation. - * + *

* To be more generic, {@link #opt(String, int)} and {@link #must(String, int)} * variants provide implementation-agnostic way to customize the builder. * Each FS-specific builder implementation can interpret the FS-specific * options accordingly, for example: - * + *

* If the option is not related to the file system, the option will be ignored. - * If the option is must, but not supported by the file system, a + * If the option is must, but not supported/known by the file system, an * {@link IllegalArgumentException} will be thrown. * */ @@ -51,10 +52,11 @@ CompletableFuture build() /** * A FileStatus may be provided to the open request. * It is up to the implementation whether to use this or not. - * @param status status. + * @param status status: may be null * @return the builder. */ - default FutureDataInputStreamBuilder withFileStatus(FileStatus status) { + default FutureDataInputStreamBuilder withFileStatus( + @Nullable FileStatus status) { return this; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/OpenFileOptions.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/OpenFileOptions.java new file mode 100644 index 0000000000000..e38eaba897b6b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/OpenFileOptions.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.hadoop.fs; + +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The standard openfile options. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class 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 seek policies: {@value}. + */ + public static final String FS_OPTION_OPENFILE_LENGTH = + FS_OPTION_OPENFILE + "length"; + + /** + * OpenFile option for seek policies: {@value}. + */ + public static final String FS_OPTION_OPENFILE_FADVISE = + FS_OPTION_OPENFILE + "fadvise"; + + /** + * fadvise policy: {@value}. + */ + public static final String FS_OPTION_OPENFILE_FADVISE_NORMAL = + "normal"; + + /** + * fadvise policy: {@value}. + */ + public static final String FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL = + "sequential"; + + /** + * fadvise policy: {@value}. + */ + public static final String FS_OPTION_OPENFILE_FADVISE_RANDOM = + "random"; + + /** + * fadvise policy: {@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. + */ + public static final Set FS_OPTION_OPENFILE_STANDARD_OPTIONS = + Stream.of( + FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_LENGTH) + .collect(Collectors.toSet()); + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java index 04c84d5595196..9790b51bd153c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.impl; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -40,16 +41,15 @@ * Builder for input streams and subclasses whose return value is * actually a completable future: this allows for better asynchronous * operation. - * + *

* To be more generic, {@link #opt(String, int)} and {@link #must(String, int)} * variants provide implementation-agnostic way to customize the builder. * Each FS-specific builder implementation can interpret the FS-specific * options accordingly, for example: - * + *

* If the option is not related to the file system, the option will be ignored. - * If the option is must, but not supported by the file system, a + * If the option is must, but not supported/known by the file system, an * {@link IllegalArgumentException} will be thrown. - * */ @InterfaceAudience.Public @InterfaceStability.Unstable @@ -147,7 +147,8 @@ public FutureDataInputStreamBuilder getThisBuilder() { } @Override - public FutureDataInputStreamBuilder withFileStatus(FileStatus st) { + public FutureDataInputStreamBuilder withFileStatus( + @Nullable FileStatus st) { this.status = st; return this; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java index 6e86f8f7afa51..ec101a438ee02 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java @@ -106,47 +106,4 @@ public FileStatus getStatus() { return status; } - - /** - * OpenFile option for seek policies: {@value}. - */ - public static final String FS_OPT_OPENFILE_FADVISE = - "fs.opt.openfile.fadvise"; - - /** - * fadvise policy: {@value}. - */ - public static final String FS_OPT_OPENFILE_FADVISE_NORMAL = "normal"; - - /** - * fadvise policy: {@value}. - */ - public static final String FS_OPT_OPENFILE_FADVISE_SEQUENTIAL = "sequential"; - - /** - * fadvise policy: {@value}. - */ - public static final String FS_OPT_OPENFILE_FADVISE_RANDOM = "random"; - - /** - * fadvise policy: {@value}. - */ - public static final String FS_OPT_OPENFILE_FADVISE_ADAPTIVE = "adaptive"; - - /** - * OpenFile option for seek policies: {@value}. - */ - public static final String FS_OPT_OPENFILE_LENGTH = - "fs.opt.openfile.length"; - - /** - * Set of standard options. - */ - public static final Set STANDARD_OPTIONS = - Stream.of( - FS_OPT_OPENFILE_FADVISE, - FS_OPT_OPENFILE_LENGTH) - .collect(Collectors.toSet()); - - } diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md index 284a964f6e522..154fcb254c612 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md @@ -746,97 +746,11 @@ exists in the metadata, but no copies of any its blocks can be located; ### `FSDataInputStreamBuilder openFile(Path path)` -Creates a [`FSDataInputStreamBuilder`](fsdatainputstreambuilder.html) -to construct a operation to open the file at `path` for reading. +See [openFile()](openfile.html). -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` -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. - ### `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` -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. +See [openFile()](openfile.html). ### `PathHandle getPathHandle(FileStatus stat, HandleOpt... options)` diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md index df538ee6cf96b..f6d5dca0d072a 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md @@ -38,3 +38,5 @@ HDFS as these are commonly expected by Hadoop client applications. 2. [Testing with the Filesystem specification](testing.html) 2. [Extending the specification and its tests](extending.html) 1. [Uploading a file using Multiple Parts](multipartuploader.html) +1. [openFile()](openfile.html). + diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md new file mode 100644 index 0000000000000..df51d043b45bb --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md @@ -0,0 +1,241 @@ + + +# `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`, +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. + + +### `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` +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. + +### `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` +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. + + +## 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. + +They MUST be viewed as hints. + +#### `fs.option.openfile.fadvise` + +Declare the read strategy of the input stream. This MAY control readahead, buffering +and other optimizations. For example, for an object store which assumes a +read is sequential, a single `GET` request may be used to read that data. +If a stream is being read in random IO, then ranged `GET` calls to blocks of +data is more efficient -so may be used instead. + +| Policy | Meaning | +| -------|---------| +| `normal` | The "Normal" policy for this store. | +| `sequential` | Optimized for sequential access. | +| `random` | Optimized purely for random seek+read/positionedRead operations. | + +Choosing the wrong read policy for an input source may be inefficient. + +Tip: log the `toString()` value of input streams at `DEBUG`. The S3A and ABFS Input +Streams log read statistics, which can provide insight about whether reads are being +performed efficiently or not. + +#### `fs.option.openfile.length` + +Declare the length of a file. This MAY be used to skip a probe for the file +length. + + +If supported by a filesystem connector, this option MUST be interpreted as declaring +the minimum length of the file: + +1. It SHALL NOT be an error if the actual length of the file is greater than this value. +1. `read()`, `seek()` and positioned read calls MAY use a position across/beyond this length + but below the actual length of the file. + Implementations MAY raise `EOFExceptions` in such cases, or they MAY return data. + + +## Example + +Here is an example from a proof of concept `org.apache.parquet.hadoop.util.HadoopInputFile` +reader which uses a (nullable) file status and a split start/end. + +The `FileStatus` value is always passed in -but if it is null, then the split +end is used to declare the length of the file. + +```java +protected SeekableInputStream newStream(Path path, FileStatus stat, + long splitStart, long splitEnd) + throws IOException { + FutureDataInputStreamBuilder builder = fs.openFile(path) + .opt("fs.option.openfile.fadvise", "random") + .withFileStatus(stat); + + // fall back to leng + if (stat == null && splitEnd > 0) { + builder.opt("fs.option.openfile.length", splitEnd); + } + CompletableFuture streamF = builder.build(); + return HadoopStreams.wrap(awaitFuture(streamF)); + } +``` + +As a result, whether driven directly by a file listing, +or when opening a file from a query plan of `(path, splitStart, splitEnd)`, +there is no need to probe the remote store for the length of the file. +When working with remote object stores, this can save tens to hundreds +of milliseconds, even if such a probe is done asynchronously. + + +## S3A Non-standard options + +The S3A Connector supports custom options + + +| 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.fadvise` | + + +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. Consult the S3A documentation. + + +## Passing options in to MapReduce + +Hadoop MapReduce will automatically read MR Job Options with the prefixes +`mapreduce.job.input.file.option.` and `mapreduce.job.input.file.must.` +prefixes, and apply these values as `.opt()` and `must()` respectively, +after remove the mapreduce-specific prefixes. + +This makes passing options in to MR jobs straightforward. +For example, to declare that a job should read its data using random IO: + +```java +JobConf jobConf = (JobConf) job.getConfiguration() +jobConf.set( + "mapreduce.job.input.file.option.fs.option.openfile.fadvise", + "random"); +``` + diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java index effae0382d9af..dd94aad17b6a1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java @@ -33,6 +33,10 @@ import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.io.IOUtils; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.contract.ContractTestUtils.compareByteArrays; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; @@ -250,7 +254,7 @@ public void testAwaitFutureTimeoutFailToFNFE() throws Throwable { public void testOpenFileExceptionallyTranslating() throws Throwable { describe("openFile missing file chains into exceptionally()"); CompletableFuture f = getFileSystem() - .openFile(path("testOpenFileUnknownOption")).build(); + .openFile(path("testOpenFileExceptionallyTranslating")).build(); interceptFuture(RuntimeException.class, "exceptionally", f.exceptionally(ex -> { @@ -262,7 +266,7 @@ public void testOpenFileExceptionallyTranslating() throws Throwable { public void testChainedFailureAwaitFuture() throws Throwable { describe("await Future handles chained failures"); CompletableFuture f = getFileSystem() - .openFile(path("testOpenFileUnknownOption")) + .openFile(path("testChainedFailureAwaitFuture")) .withFileStatus(null) .build(); intercept(RuntimeException.class, @@ -320,17 +324,43 @@ public void testOpenFileApplyAsyncRead() throws Throwable { dataset(4, 0x40, 0x80)); CompletableFuture future = fs.openFile(path).build(); AtomicBoolean accepted = new AtomicBoolean(false); - future.thenAcceptAsync(i -> accepted.set(true)).get(); + future.thenApply(stream -> { + accepted.set(true); + return stream; + }).get().close(); assertTrue("async accept operation not invoked", accepted.get()); } + /** + * Open a file with a null status, and the length + * passed in as a must() option (along with sequential IO). + * The file is opened, the data read, and it must match + * the source data. + */ @Test public void testOpenFileNullStatus() throws Throwable { - describe("use openFile() with a null status"); + describe("use openFile() with a null status and expect the status to be" + + " ignored"); Path path = path("testOpenFileNullStatus"); - intercept(NullPointerException.class, - () -> getFileSystem().openFile(path).withFileStatus(null)); + FileSystem fs = getFileSystem(); + int len = 4; + byte[] result = new byte[len]; + byte[] dataset = dataset(len, 0x40, 0x80); + createFile(fs, path, true, + dataset); + CompletableFuture future = fs.openFile(path) + .withFileStatus(null) + .must(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .must(FS_OPTION_OPENFILE_LENGTH, + len) + .build(); + + try (FSDataInputStream in = future.get();) { + in.read(result, 0, len); + } + compareByteArrays(dataset, result, len); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 80959e8596f19..3da528f625a5b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -24,8 +24,6 @@ import java.util.concurrent.TimeUnit; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; - /** * All the constants used with the {@link S3AFileSystem}. * diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index f046126af9d31..b27e614c538cf 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -107,10 +107,9 @@ import org.apache.hadoop.fs.s3a.impl.DeleteOperation; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl; -import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; -import org.apache.hadoop.fs.s3a.impl.S3AOpenFileHelper; +import org.apache.hadoop.fs.s3a.impl.S3AOpenFileOperation; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; import org.apache.hadoop.fs.s3a.impl.RenameOperation; import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder; @@ -119,7 +118,6 @@ import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder; import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatisticsImpl; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; -import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; import org.apache.hadoop.io.IOUtils; @@ -306,7 +304,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, /** * Helper for the openFile() method. */ - private S3AOpenFileHelper openFileHelper; + private S3AOpenFileOperation openFileHelper; /** * Directory policy. @@ -486,7 +484,7 @@ public void initialize(URI name, Configuration originalConf) BULK_DELETE_PAGE_SIZE_DEFAULT, 0); listing = new Listing(listingOperationCallbacks, createStoreContext()); // now the open file logic - openFileHelper = new S3AOpenFileHelper( + openFileHelper = new S3AOpenFileOperation( inputPolicy, changeDetectionPolicy, readAhead, @@ -1129,7 +1127,7 @@ public FSDataInputStream open(Path f, int bufferSize) @Retries.RetryTranslated private FSDataInputStream open( final Path path, - final S3AOpenFileHelper.OpenFileInformation fileInformation) + final S3AOpenFileOperation.OpenFileInformation fileInformation) throws IOException { final S3AFileStatus fileStatus = @@ -4729,7 +4727,7 @@ protected void setTtlTimeProvider(ITtlTimeProvider ttlTimeProvider) { @Retries.RetryTranslated private FSDataInputStream select(final Path path, final Configuration options, - final S3AOpenFileHelper.OpenFileInformation fileInformation) + final S3AOpenFileOperation.OpenFileInformation fileInformation) throws IOException { entryPoint(OBJECT_SELECT_REQUESTS); requireSelectSupport(path); @@ -4802,7 +4800,7 @@ private void requireSelectSupport(final Path source) throws */ private S3AFileStatus extractOrFetchSimpleFileStatus( final Path path, - final S3AOpenFileHelper.OpenFileInformation fileInformation) + final S3AOpenFileOperation.OpenFileInformation fileInformation) throws IOException { S3AFileStatus fileStatus = fileInformation.getStatus(); if (fileStatus == null) { @@ -4838,7 +4836,7 @@ public CompletableFuture openFileWithOptions( final Path rawPath, final OpenFileParameters parameters) throws IOException { final Path path = qualify(rawPath); - S3AOpenFileHelper.OpenFileInformation fileInformation = + S3AOpenFileOperation.OpenFileInformation fileInformation = openFileHelper.prepareToOpenFile( path, parameters, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java index bc726fea4eb95..c9690df25ad67 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java @@ -25,7 +25,7 @@ import java.util.Locale; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE_ADAPTIVE; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_ADAPTIVE; import static org.apache.hadoop.fs.s3a.Constants.*; /** @@ -63,7 +63,7 @@ public static S3AInputPolicy getPolicy(String name) { String trimmed = name.trim().toLowerCase(Locale.ENGLISH); switch (trimmed) { case INPUT_FADV_NORMAL: - case FS_OPT_OPENFILE_FADVISE_ADAPTIVE: + case FS_OPTION_OPENFILE_FADVISE_ADAPTIVE: return Normal; case INPUT_FADV_RANDOM: return Random; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java index 98b76b15da48f..aec3bb56d7340 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AbstractStoreOperation.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.s3a.impl; -import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; +import javax.annotation.Nullable; /** * Base class of operations in the store. @@ -34,8 +34,8 @@ public abstract class AbstractStoreOperation { * constructor. * @param storeContext store context. */ - protected AbstractStoreOperation(final StoreContext storeContext) { - this.storeContext = checkNotNull(storeContext); + protected AbstractStoreOperation(final @Nullable StoreContext storeContext) { + this.storeContext = storeContext; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index ee09c9cef5c14..11181137bbcaa 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -27,8 +27,8 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.s3a.Constants; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; /** * Internal constants private only to the S3A codebase. @@ -83,8 +83,8 @@ private InternalConstants() { @InterfaceStability.Unstable public static final Set S3A_OPENFILE_KEYS = Stream.of( - FS_OPT_OPENFILE_FADVISE, - FS_OPT_OPENFILE_LENGTH, + FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_LENGTH, Constants.INPUT_FADVISE, Constants.READAHEAD_RANGE) .collect(Collectors.toSet()); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java similarity index 94% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileHelper.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java index bb467613e29ad..f41bf67f9c7c1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java @@ -37,8 +37,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; @@ -49,10 +49,10 @@ * This got complex enough it merited removal from S3AFS -which * also permits unit testing. */ - public class S3AOpenFileHelper { + public class S3AOpenFileOperation extends AbstractStoreOperation { private static final Logger LOG = - LoggerFactory.getLogger(S3AOpenFileHelper.class); + LoggerFactory.getLogger(S3AOpenFileOperation.class); private final S3AInputPolicy inputPolicy; @@ -76,11 +76,12 @@ public class S3AOpenFileHelper { * @param readAheadRange read ahead range * @param username username */ - public S3AOpenFileHelper( + public S3AOpenFileOperation( final S3AInputPolicy inputPolicy, final ChangeDetectionPolicy changePolicy, final long readAheadRange, final String username) { + super(null); this.inputPolicy = inputPolicy; this.changePolicy = changePolicy; this.readAheadRange = readAheadRange; @@ -249,10 +250,10 @@ public OpenFileInformation prepareToOpenFile( username, eTag, versionId); - } else if (options.get(FS_OPT_OPENFILE_LENGTH) != null) { + } else if (options.get(FS_OPTION_OPENFILE_LENGTH) != null) { // build a minimal S3A FileStatus From the input length alone. // this is all we actually need. - long length = options.getLong(FS_OPT_OPENFILE_LENGTH, 0); + long length = options.getLong(FS_OPTION_OPENFILE_LENGTH, 0); LOG.debug("Fixing length of file to read {} as {}", path, length); fileStatus = new S3AFileStatus( length, @@ -267,7 +268,7 @@ public OpenFileInformation prepareToOpenFile( } // seek policy from default, s3a opt or standard option String policy1 = options.get(INPUT_FADVISE, inputPolicy.toString()); - String policy2 = options.get(FS_OPT_OPENFILE_FADVISE, policy1); + String policy2 = options.get(FS_OPTION_OPENFILE_FADVISE, policy1); S3AInputPolicy policy = S3AInputPolicy.getPolicy(policy2); // readahead range diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java index 66ec8ff3a22dc..04ff897809781 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java @@ -497,7 +497,9 @@ public void testOpenFileWithStatus() throws Throwable { S3AFileStatus s2 = new S3AFileStatus(true, testpath, "alice"); assertTrue("not a directory " + s2, s2.isDirectory()); LOG.info("Open with directory status"); - interceptFuture(FileNotFoundException.class, "", + // rejection is currently during the synchronous phase + // because the + intercept(FileNotFoundException.class, "", () -> fs.openFile(testpath) .withFileStatus(s2) .build()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java index f3ac569843454..bcd8ae0377406 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java @@ -34,15 +34,15 @@ import org.apache.hadoop.test.HadoopTestBase; import static java.util.Collections.singleton; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE_ADAPTIVE; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE_RANDOM; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_ADAPTIVE; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_RANDOM; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** - * Unit tests for {@link S3AOpenFileHelper}. + * Unit tests for {@link S3AOpenFileOperation}. */ public class TestOpenFileHelper extends HadoopTestBase { @@ -60,17 +60,17 @@ public class TestOpenFileHelper extends HadoopTestBase { private static final Path TESTPATH = new Path(TESTFILE); - private S3AOpenFileHelper helper; + private S3AOpenFileOperation helper; @Before public void setup() { - helper = new S3AOpenFileHelper(INPUT_POLICY, + helper = new S3AOpenFileOperation(INPUT_POLICY, CHANGE_POLICY, READ_AHEAD_RANGE, USERNAME); } @Test public void testSimpleFile() throws Throwable { - ObjectAssert + ObjectAssert asst = assertFI(helper.openSimpleFile()); asst.extracting(f -> f.getChangePolicy()) @@ -81,8 +81,8 @@ public void testSimpleFile() throws Throwable { .isEqualTo(READ_AHEAD_RANGE); } - private ObjectAssert assertFI( - final S3AOpenFileHelper.OpenFileInformation fi) { + private ObjectAssert assertFI( + final S3AOpenFileOperation.OpenFileInformation fi) { return Assertions.assertThat(fi) .describedAs("File Information %s", fi); } @@ -101,9 +101,9 @@ public void testUnknownMandatory() throws Throwable { public void testSeekPolicy() throws Throwable { // ask for random IO - ObjectAssert asst = + ObjectAssert asst = assertFI(helper.prepareToOpenFile(TESTPATH, - params(INPUT_FADVISE, FS_OPT_OPENFILE_FADVISE_RANDOM), + params(INPUT_FADVISE, FS_OPTION_OPENFILE_FADVISE_RANDOM), 0)); // is picked up @@ -118,9 +118,9 @@ public void testSeekPolicy() throws Throwable { public void testSeekPolicyAdaptive() throws Throwable { // when caller asks for adaptive, they get "normal" - ObjectAssert asst = + ObjectAssert asst = assertFI(helper.prepareToOpenFile(TESTPATH, - params(INPUT_FADVISE, FS_OPT_OPENFILE_FADVISE_ADAPTIVE), + params(INPUT_FADVISE, FS_OPTION_OPENFILE_FADVISE_ADAPTIVE), 0)); asst.extracting(f -> f.getInputPolicy()) @@ -131,7 +131,7 @@ public void testSeekPolicyAdaptive() throws Throwable { public void testUnknownSeekPolicy() throws Throwable { // fall back to the normal seek policy. - ObjectAssert asst = + ObjectAssert asst = assertFI(helper.prepareToOpenFile(TESTPATH, params(INPUT_FADVISE, "undefined"), 0)); @@ -144,7 +144,7 @@ public void testUnknownSeekPolicy() throws Throwable { public void testReadahead() throws Throwable { // readahead range option - ObjectAssert asst = + ObjectAssert asst = assertFI(helper.prepareToOpenFile(TESTPATH, params(READAHEAD_RANGE, "4096"), 0)); @@ -156,9 +156,9 @@ public void testReadahead() throws Throwable { @Test public void testStatusWithValidFilename() throws Throwable { Path p = new Path("file:///tmp/" + TESTPATH.getName()); - ObjectAssert asst = + ObjectAssert asst = assertFI(helper.prepareToOpenFile(TESTPATH, - params(FS_OPT_OPENFILE_LENGTH, "32") + params(FS_OPTION_OPENFILE_LENGTH, "32") .withStatus(status(p, 4096)), 0)); asst.extracting(f -> f.getStatus().getVersionId()) @@ -172,9 +172,9 @@ public void testStatusWithValidFilename() throws Throwable { @Test public void testLocatedStatus() throws Throwable { Path p = new Path("file:///tmp/" + TESTPATH.getName()); - ObjectAssert asst = + ObjectAssert asst = assertFI(helper.prepareToOpenFile(TESTPATH, - params(FS_OPT_OPENFILE_LENGTH, "32") + params(FS_OPTION_OPENFILE_LENGTH, "32") .withStatus( new S3ALocatedFileStatus( status(p, 4096), null)), @@ -217,9 +217,9 @@ public void testStatusWithInconsistentFilename() throws Throwable { */ @Test public void testFileLength() throws Throwable { - ObjectAssert asst = + ObjectAssert asst = assertFI(helper.prepareToOpenFile(TESTPATH, - params(FS_OPT_OPENFILE_LENGTH, "8192") + params(FS_OPTION_OPENFILE_LENGTH, "8192") .withStatus(null), 0)); asst.extracting(f -> f.getStatus()) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java index 51c34a81ee903..0b528fbafb651 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java @@ -38,10 +38,9 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.readStream; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_FADVISE_SEQUENTIAL; -import static org.apache.hadoop.fs.impl.OpenFileParameters.FS_OPT_OPENFILE_LENGTH; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_CLOSE_BYTES_READ; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_OPENED; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_SEEK_BYTES_SKIPPED; @@ -131,14 +130,14 @@ public void testOpenFileCost() throws Throwable { int offset = 2; long shortLen = len - offset; CompletableFuture f2 = fs.openFile(testFile) - .must(FS_OPT_OPENFILE_FADVISE, FS_OPT_OPENFILE_FADVISE_SEQUENTIAL) - .opt(FS_OPT_OPENFILE_LENGTH, shortLen) + .must(FS_OPTION_OPENFILE_FADVISE, FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .opt(FS_OPTION_OPENFILE_LENGTH, shortLen) .build(); FSDataInputStream in2 = verifyMetrics(() -> fs.openFile(testFile) - .must(FS_OPT_OPENFILE_FADVISE, - FS_OPT_OPENFILE_FADVISE_SEQUENTIAL) - .opt(FS_OPT_OPENFILE_LENGTH, shortLen) + .must(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .opt(FS_OPTION_OPENFILE_LENGTH, shortLen) .build() .get(), always(NO_IO), @@ -158,8 +157,9 @@ public void testOpenFileCost() throws Throwable { long longLen = len + 10; FSDataInputStream in3 = verifyMetrics(() -> fs.openFile(testFile) - .must(FS_OPT_OPENFILE_FADVISE, FS_OPT_OPENFILE_FADVISE_SEQUENTIAL) - .must(FS_OPT_OPENFILE_LENGTH, longLen) + .must(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .must(FS_OPTION_OPENFILE_LENGTH, longLen) .build() .get(), always(NO_IO)); From 72437dec44a6e89d255941841f7fd48bca8b0528 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 18 Sep 2020 19:45:06 +0100 Subject: [PATCH 04/10] HADOOP-16202: openFile() - move all options into a class nested under o.a.h.fs.Options; - use in FsShell commands which download files (copy, head, tail) - distcp download - FileUtil.copy Includes YARN-10444 changes in yarn localization. This ensures that if the default/cluster settings For a file system are optimized for random IO, artifact downloads are still read at the maximum speed possible (one big GET to the EOF). In the changed IOUtils and YARN Code, also tweaked slightly * use incremental listing for dir listing, better for paged loads * pass around a FileStatus more Change-Id: Iedbb7b53cdaef2b82a0876e1db379c9a428f9339 --- .../apache/hadoop/fs/ChecksumFileSystem.java | 4 +- .../java/org/apache/hadoop/fs/FileSystem.java | 2 +- .../java/org/apache/hadoop/fs/FileUtil.java | 55 ++++++-- .../org/apache/hadoop/fs/OpenFileOptions.java | 92 ------------- .../java/org/apache/hadoop/fs/Options.java | 72 ++++++++++ .../hadoop/fs/impl/OpenFileParameters.java | 2 - .../apache/hadoop/fs/shell/CopyCommands.java | 3 +- .../org/apache/hadoop/fs/shell/Display.java | 3 +- .../java/org/apache/hadoop/fs/shell/Head.java | 8 +- .../org/apache/hadoop/fs/shell/PathData.java | 36 +++++ .../java/org/apache/hadoop/fs/shell/Tail.java | 11 +- .../apache/hadoop/util/JsonSerialization.java | 11 +- .../fs/contract/AbstractContractOpenTest.java | 12 +- .../apache/hadoop/fs/s3a/S3AInputPolicy.java | 2 +- .../hadoop/fs/s3a/impl/InternalConstants.java | 4 +- .../fs/s3a/impl/S3AOpenFileOperation.java | 19 +-- .../fs/s3a/impl/TestOpenFileHelper.java | 130 +++++++++++------- .../fs/s3a/performance/ITestS3AOpenCost.java | 87 +++++++----- .../mapred/RetriableFileCopyCommand.java | 8 +- .../apache/hadoop/yarn/util/FSDownload.java | 24 +++- 20 files changed, 360 insertions(+), 225 deletions(-) delete mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/OpenFileOptions.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java index f081742ce59db..c9950a95abac8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java @@ -24,7 +24,6 @@ import java.io.InputStream; import java.nio.channels.ClosedChannelException; import java.util.Arrays; -import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -42,6 +41,7 @@ import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.util.Progressable; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; /**************************************************************** @@ -847,7 +847,7 @@ protected CompletableFuture openFileWithOptions( final OpenFileParameters parameters) throws IOException { AbstractFSBuilderImpl.rejectUnknownMandatoryKeys( parameters.getMandatoryKeys(), - Collections.emptySet(), + FS_OPTION_OPENFILE_STANDARD_OPTIONS, "for " + path); return LambdaUtils.eval( new CompletableFuture<>(), diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index 940f7dbeb3d6d..4a6729ed56dd9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -4655,7 +4655,7 @@ protected CompletableFuture openFileWithOptions( final OpenFileParameters parameters) throws IOException { AbstractFSBuilderImpl.rejectUnknownMandatoryKeys( parameters.getMandatoryKeys(), - OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS, ""); + Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS, ""); CompletableFuture result = new CompletableFuture<>(); try { result.complete(open(pathHandle, parameters.getBufferSize())); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java index e078a2c519621..6f5e8f445833c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java @@ -71,6 +71,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; + /** * A collection of file-processing util methods */ @@ -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. + *

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

+ * 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"). + *

+ * 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 + */ public static boolean copy(FileSystem srcFS, FileStatus srcStatus, FileSystem dstFS, Path dst, boolean deleteSource, @@ -409,22 +439,27 @@ public static boolean copy(FileSystem srcFS, FileStatus srcStatus, if (!dstFS.mkdirs(dst)) { return false; } - FileStatus contents[] = srcFS.listStatus(src); - for (int i = 0; i < contents.length; i++) { - copy(srcFS, contents[i], dstFS, - new Path(dst, contents[i].getPath().getName()), - deleteSource, overwrite, conf); + RemoteIterator contents = srcFS.listStatusIterator(src); + while (contents.hasNext()) { + FileStatus next = contents.next(); + copy(srcFS, next, dstFS, + new Path(dst, next.getPath().getName()), + deleteSource, overwrite, conf); } } else { - InputStream in=null; + InputStream in = null; OutputStream out = null; try { - in = srcFS.open(src); + in = awaitFuture(srcFS.openFile(src) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .opt(FS_OPTION_OPENFILE_LENGTH, + srcStatus.getLen()) // file length hint for object stores + .build()); out = dstFS.create(dst, overwrite); IOUtils.copyBytes(in, out, conf, true); } catch (IOException e) { - IOUtils.closeStream(out); - IOUtils.closeStream(in); + IOUtils.cleanupWithLogger(LOG, in, out); throw e; } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/OpenFileOptions.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/OpenFileOptions.java deleted file mode 100644 index e38eaba897b6b..0000000000000 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/OpenFileOptions.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ - -package org.apache.hadoop.fs; - -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * The standard openfile options. - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public class 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 seek policies: {@value}. - */ - public static final String FS_OPTION_OPENFILE_LENGTH = - FS_OPTION_OPENFILE + "length"; - - /** - * OpenFile option for seek policies: {@value}. - */ - public static final String FS_OPTION_OPENFILE_FADVISE = - FS_OPTION_OPENFILE + "fadvise"; - - /** - * fadvise policy: {@value}. - */ - public static final String FS_OPTION_OPENFILE_FADVISE_NORMAL = - "normal"; - - /** - * fadvise policy: {@value}. - */ - public static final String FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL = - "sequential"; - - /** - * fadvise policy: {@value}. - */ - public static final String FS_OPTION_OPENFILE_FADVISE_RANDOM = - "random"; - - /** - * fadvise policy: {@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. - */ - public static final Set FS_OPTION_OPENFILE_STANDARD_OPTIONS = - Stream.of( - FS_OPTION_OPENFILE_FADVISE, - FS_OPTION_OPENFILE_LENGTH) - .collect(Collectors.toSet()); - -} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java index 75bc12df8fdcf..9d8be90a30d62 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java @@ -18,8 +18,11 @@ package org.apache.hadoop.fs; import java.util.Optional; +import java.util.Set; import java.util.function.Function; import java.util.function.BiFunction; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -518,4 +521,73 @@ public enum ChecksumCombineMode { 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 seek policies: {@value}. + */ + public static final String FS_OPTION_OPENFILE_LENGTH = + FS_OPTION_OPENFILE + "length"; + + /** + * 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. + */ + public static final Set FS_OPTION_OPENFILE_STANDARD_OPTIONS = + Stream.of( + FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_LENGTH) + .collect(Collectors.toSet()); + + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java index ec101a438ee02..a19c5faff4d90 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/OpenFileParameters.java @@ -19,8 +19,6 @@ package org.apache.hadoop.fs.impl; import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java index 77f63170593ab..980d7f704b5a4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java @@ -102,7 +102,8 @@ protected void processArguments(LinkedList items) try { for (PathData src : srcs) { if (src.stat.getLen() != 0) { - try (FSDataInputStream in = src.fs.open(src.path)) { + // Always do sequential reads. + try (FSDataInputStream in = src.openForSequentialIO()) { IOUtils.copyBytes(in, out, getConf(), false); writeDelimiter(out); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java index 670fa152f72ed..d3ca013a3f251 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Display.java @@ -105,7 +105,8 @@ private void printToStdout(InputStream in) throws IOException { } protected InputStream getInputStream(PathData item) throws IOException { - return item.fs.open(item.path); + // Always do sequential reads; + return item.openForSequentialIO(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Head.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Head.java index 2280225b5ae32..ddd26ff9a69ec 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Head.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Head.java @@ -28,6 +28,8 @@ import java.util.LinkedList; import java.util.List; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_NORMAL; + /** * Show the first 1KB of the file. */ @@ -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)) { IOUtils.copyBytes(in, System.out, endingOffset, false); - } finally { - in.close(); } } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java index dad54ea07bdf1..260a775cd41d6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java @@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; @@ -39,6 +40,11 @@ import org.apache.hadoop.fs.PathNotFoundException; import org.apache.hadoop.fs.RemoteIterator; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; + /** * Encapsulates a Path (path), its FileStatus (stat), and its FileSystem (fs). * PathData ensures that the returned path string will be the same as the @@ -611,4 +617,34 @@ public boolean equals(Object o) { public int hashCode() { return path.hashCode(); } + + + /** + * Open a file for sequential IO. + *

+ * This uses FileSystem.openFile() to request sequential IO; + * the file status is also passed in. + * Filesystems may use to optimize their IO. + * @return an input stream + * @throws IOException failure + */ + protected FSDataInputStream openForSequentialIO() + throws IOException { + return openFile(FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL); + } + + /** + * Open a file. + * @param policy fadvise policy. + * @return an input stream + * @throws IOException failure + */ + protected FSDataInputStream openFile(final String policy) throws IOException { + return awaitFuture(fs.openFile(path) + .opt(FS_OPTION_OPENFILE_FADVISE, + policy) + .opt(FS_OPTION_OPENFILE_LENGTH, + stat.getLen()) // file length hint for object stores + .build()); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java index 22dd32bce8512..f928d560b79b9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Tail.java @@ -30,6 +30,8 @@ import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_NORMAL; + /** * Get a listing of all files in that match the file patterns. */ @@ -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. + try (FSDataInputStream in = item.openFile( + FS_OPTION_OPENFILE_FADVISE_NORMAL)) { in.seek(offset); // use conf so the system configured io block size is used IOUtils.copyBytes(in, System.out, getConf(), false); offset = in.getPos(); - } finally { - in.close(); } return offset; } + } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java index 7a26ecadee456..4b4ea6b415d10 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java @@ -46,6 +46,10 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; + /** * Support for marshalling objects to and from JSON. * @@ -243,7 +247,12 @@ public T fromInstance(T instance) throws IOException { * @throws IOException IO or JSON parse problems */ public T load(FileSystem fs, Path path) throws IOException { - try (FSDataInputStream dataInputStream = fs.open(path)) { + + try (FSDataInputStream dataInputStream = + awaitFuture(fs.openFile(path) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .build())) { // throw an EOF exception if there is no data available. if (dataInputStream.available() == 0) { throw new EOFException("No data in " + path); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java index dd94aad17b6a1..cf6dc5571ec55 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java @@ -33,9 +33,9 @@ import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.io.IOUtils; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.contract.ContractTestUtils.compareByteArrays; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; @@ -339,9 +339,9 @@ public void testOpenFileApplyAsyncRead() throws Throwable { * the source data. */ @Test - public void testOpenFileNullStatus() throws Throwable { + public void testOpenFileNullStatusButFileLength() throws Throwable { describe("use openFile() with a null status and expect the status to be" - + " ignored"); + + " ignored. fadvise and length are passed in as must() options"); Path path = path("testOpenFileNullStatus"); FileSystem fs = getFileSystem(); int len = 4; @@ -357,7 +357,7 @@ public void testOpenFileNullStatus() throws Throwable { len) .build(); - try (FSDataInputStream in = future.get();) { + try (FSDataInputStream in = future.get()) { in.read(result, 0, len); } compareByteArrays(dataset, result, len); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java index c9690df25ad67..eb3539dd076c9 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java @@ -25,7 +25,7 @@ import java.util.Locale; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_ADAPTIVE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_ADAPTIVE; import static org.apache.hadoop.fs.s3a.Constants.*; /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 11181137bbcaa..73d13ca7e77b8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -27,8 +27,8 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.s3a.Constants; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; /** * Internal constants private only to the S3A codebase. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java index f41bf67f9c7c1..89b87582b35c4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java @@ -36,9 +36,9 @@ import org.apache.hadoop.fs.s3a.select.SelectConstants; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; @@ -49,7 +49,7 @@ * This got complex enough it merited removal from S3AFS -which * also permits unit testing. */ - public class S3AOpenFileOperation extends AbstractStoreOperation { +public class S3AOpenFileOperation extends AbstractStoreOperation { private static final Logger LOG = LoggerFactory.getLogger(S3AOpenFileOperation.class); @@ -68,7 +68,6 @@ public class S3AOpenFileOperation extends AbstractStoreOperation { */ private final OpenFileInformation simpleFileInformation; - /** * Instantiate with the default options from the filesystem. * @param inputPolicy input policy @@ -103,7 +102,7 @@ public static final class OpenFileInformation { /** File status; may be null. */ private final S3AFileStatus status; - /** SQL string if this is a SQL select file*/ + /** SQL string if this is a SQL select file. */ private final String sql; /** Active input policy. */ @@ -115,6 +114,9 @@ public static final class OpenFileInformation { /** read ahead range. */ private final long readAheadRange; + /** + * Constructor. + */ private OpenFileInformation(final boolean isSql, final S3AFileStatus status, final String sql, @@ -221,7 +223,7 @@ public OpenFileInformation prepareToOpenFile( String eTag; // can use this status to skip our own probes, LOG.debug("File was opened with a supplied FileStatus;" - + " skipping getFileStatus call in open() operation: {}", + + " skipping getFileStatus call in open() operation: {}", providedStatus); if (providedStatus instanceof S3AFileStatus) { @@ -261,7 +263,8 @@ public OpenFileInformation prepareToOpenFile( path, blockSize, username, - null, null); + null, + null); } else { // neither a file status nor a file length fileStatus = null; @@ -278,7 +281,7 @@ public OpenFileInformation prepareToOpenFile( } /** - * Open a simple file by builing the base + * Open a simple file. * @return the parameters needed to open a file through open(). */ public OpenFileInformation openSimpleFile() { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java index bcd8ae0377406..494bb70e9e2b7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.impl; import java.io.FileNotFoundException; +import java.io.IOException; import org.assertj.core.api.Assertions; import org.assertj.core.api.ObjectAssert; @@ -34,9 +35,9 @@ import org.apache.hadoop.test.HadoopTestBase; import static java.util.Collections.singleton; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_ADAPTIVE; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_RANDOM; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_ADAPTIVE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_RANDOM; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -81,75 +82,79 @@ public void testSimpleFile() throws Throwable { .isEqualTo(READ_AHEAD_RANGE); } + /** + * Initiate an assert from an open file information instance. + * @param fi file info + * @return an assert stream. + */ private ObjectAssert assertFI( final S3AOpenFileOperation.OpenFileInformation fi) { return Assertions.assertThat(fi) .describedAs("File Information %s", fi); } + public ObjectAssert assertOpenFile( + final String key, final String option) throws IOException { + return assertFI(prepareToOpenFile(params(key, option))); + } + @Test - public void testUnknownMandatory() throws Throwable { + public void testUnknownMandatoryOption() throws Throwable { String key = "unknown"; intercept(IllegalArgumentException.class, key, () -> - helper.prepareToOpenFile(TESTPATH, - params(key, "undefined"), - 0)); + prepareToOpenFile(params(key, "undefined"))); } @Test - public void testSeekPolicy() throws Throwable { + public void testSeekRandomIOPolicy() throws Throwable { // ask for random IO - ObjectAssert asst = - assertFI(helper.prepareToOpenFile(TESTPATH, - params(INPUT_FADVISE, FS_OPTION_OPENFILE_FADVISE_RANDOM), - 0)); + String option = FS_OPTION_OPENFILE_FADVISE_RANDOM; // is picked up - asst.extracting(f -> f.getInputPolicy()) + assertOpenFile(INPUT_FADVISE, option).extracting(f -> f.getInputPolicy()) .isEqualTo(S3AInputPolicy.Random); // and as neither status nor length was set: no file status - asst.extracting(f -> f.getStatus()) + assertOpenFile(INPUT_FADVISE, option).extracting(f -> f.getStatus()) .isNull(); } + @Test public void testSeekPolicyAdaptive() throws Throwable { // when caller asks for adaptive, they get "normal" ObjectAssert asst = - assertFI(helper.prepareToOpenFile(TESTPATH, - params(INPUT_FADVISE, FS_OPTION_OPENFILE_FADVISE_ADAPTIVE), - 0)); + assertOpenFile(INPUT_FADVISE, FS_OPTION_OPENFILE_FADVISE_ADAPTIVE); asst.extracting(f -> f.getInputPolicy()) .isEqualTo(S3AInputPolicy.Normal); } + /** + * Verify that an unknown seek policy falls back to "normal". + */ @Test public void testUnknownSeekPolicy() throws Throwable { // fall back to the normal seek policy. - ObjectAssert asst = - assertFI(helper.prepareToOpenFile(TESTPATH, - params(INPUT_FADVISE, "undefined"), - 0)); - asst.extracting(f -> f.getInputPolicy()) + assertOpenFile(INPUT_FADVISE, "undefined") + .extracting(f -> f.getInputPolicy()) .isEqualTo(S3AInputPolicy.Normal); } + /** + * Verify readahead range is picked up. + */ @Test public void testReadahead() throws Throwable { // readahead range option - ObjectAssert asst = - assertFI(helper.prepareToOpenFile(TESTPATH, - params(READAHEAD_RANGE, "4096"), - 0)); - asst.extracting(f -> f.getReadAheadRange()) + assertOpenFile(READAHEAD_RANGE, "4096") + .extracting(f -> f.getReadAheadRange()) .isEqualTo(4096L); } @@ -157,10 +162,9 @@ public void testReadahead() throws Throwable { public void testStatusWithValidFilename() throws Throwable { Path p = new Path("file:///tmp/" + TESTPATH.getName()); ObjectAssert asst = - assertFI(helper.prepareToOpenFile(TESTPATH, + assertFI(prepareToOpenFile( params(FS_OPTION_OPENFILE_LENGTH, "32") - .withStatus(status(p, 4096)), - 0)); + .withStatus(status(p, 4096)))); asst.extracting(f -> f.getStatus().getVersionId()) .isEqualTo("version"); asst.extracting(f -> f.getStatus().getETag()) @@ -169,16 +173,19 @@ public void testStatusWithValidFilename() throws Throwable { .isEqualTo(4096L); } + /** + * Verify S3ALocatedFileStatus is handled. + */ @Test public void testLocatedStatus() throws Throwable { Path p = new Path("file:///tmp/" + TESTPATH.getName()); ObjectAssert asst = - assertFI(helper.prepareToOpenFile(TESTPATH, - params(FS_OPTION_OPENFILE_LENGTH, "32") - .withStatus( - new S3ALocatedFileStatus( - status(p, 4096), null)), - 0)); + assertFI( + prepareToOpenFile( + params(FS_OPTION_OPENFILE_LENGTH, "32") + .withStatus( + new S3ALocatedFileStatus( + status(p, 4096), null)))); asst.extracting(f -> f.getStatus().getVersionId()) .isEqualTo("version"); asst.extracting(f -> f.getStatus().getETag()) @@ -193,10 +200,9 @@ public void testLocatedStatus() throws Throwable { @Test public void testDirectoryStatus() throws Throwable { intercept(FileNotFoundException.class, TESTFILE, () -> - helper.prepareToOpenFile(TESTPATH, + prepareToOpenFile( params(INPUT_FADVISE, "normal") - .withStatus(new S3AFileStatus(true, TESTPATH, USERNAME)), - 0)); + .withStatus(new S3AFileStatus(true, TESTPATH, USERNAME)))); } /** @@ -205,23 +211,28 @@ public void testDirectoryStatus() throws Throwable { @Test public void testStatusWithInconsistentFilename() throws Throwable { intercept(IllegalArgumentException.class, TESTFILE, () -> - helper.prepareToOpenFile(TESTPATH, - params(INPUT_FADVISE, "normal") - .withStatus(new S3AFileStatus(true, - new Path(TESTFILE + "-"), USERNAME)), - 0)); + prepareToOpenFile(params(INPUT_FADVISE, "normal") + .withStatus(new S3AFileStatus(true, + new Path(TESTFILE + "-"), USERNAME)))); + } + + public S3AOpenFileOperation.OpenFileInformation prepareToOpenFile( + final OpenFileParameters parameters) + throws IOException { + return helper.prepareToOpenFile(TESTPATH, + parameters, 0); } /** - * If a file length option is set, create a file status + * If a file length option is set, a file status + * is created. */ @Test public void testFileLength() throws Throwable { ObjectAssert asst = - assertFI(helper.prepareToOpenFile(TESTPATH, + assertFI(prepareToOpenFile( params(FS_OPTION_OPENFILE_LENGTH, "8192") - .withStatus(null), - 0)); + .withStatus(null))); asst.extracting(f -> f.getStatus()) .isNotNull(); asst.extracting(f -> f.getStatus().getPath()) @@ -231,17 +242,36 @@ public void testFileLength() throws Throwable { } - private S3AFileStatus status(final Path p, final int length) { + /** + * Create an S3A status entry with stub etag and versions, timestamp of 0. + * @param path status path + * @param length file length + * @return a status instance. + */ + private S3AFileStatus status(final Path path, final int length) { return new S3AFileStatus(length, 0, - p, 0, "", "etag", "version"); + path, 0, "", "etag", "version"); } + /** + * Create an instance of {@link OpenFileParameters} with + * the key as a mandatory parameter. + * @param key mandatory key + * @param val value + * @return the instance. + */ private OpenFileParameters params(final String key, final String val) { return new OpenFileParameters() .withMandatoryKeys(singleton(key)) .withOptions(conf(key, val)); } + /** + * Create a configuration with a single entry. + * @param key entry key + * @param val entry value + * @return a configuration + */ private Configuration conf(String key, Object val) { Configuration c = new Configuration(false); c.set(key, val.toString()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java index 0b528fbafb651..10703e0c44004 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java @@ -22,7 +22,6 @@ import java.io.EOFException; import java.util.Arrays; import java.util.Collection; -import java.util.concurrent.CompletableFuture; import org.junit.Test; import org.junit.runner.RunWith; @@ -36,11 +35,11 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.contract.ContractTestUtils.readStream; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; -import static org.apache.hadoop.fs.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_CLOSE_BYTES_READ; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_OPENED; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_SEEK_BYTES_SKIPPED; @@ -56,6 +55,12 @@ public class ITestS3AOpenCost extends AbstractS3ACostTest { private static final Logger LOG = LoggerFactory.getLogger(ITestS3AOpenCost.class); + private Path testFile; + + private FileStatus testFileStatus; + + private long fileLength; + /** * Parameterization. */ @@ -73,6 +78,21 @@ public ITestS3AOpenCost(final String name, super(s3guard, keepMarkers, authoritative); } + /** + * Setup creates a test file, saves is status and length + * to fields. + */ + @Override + public void setup() throws Exception { + super.setup(); + S3AFileSystem fs = getFileSystem(); + testFile = methodPath(); + + writeTextFile(fs, testFile, "openfile", true); + testFileStatus = fs.getFileStatus(testFile); + fileLength = testFileStatus.getLen(); + } + /** * Test when openFile() performs GET requests when file status * and length options are passed down. @@ -82,57 +102,53 @@ public ITestS3AOpenCost(final String name, * This is slightly less than ideal. */ @Test - public void testOpenFileCost() throws Throwable { + public void testOpenFileWithStatusOfOtherFS() throws Throwable { describe("Test cost of openFile with/without status; raw only"); S3AFileSystem fs = getFileSystem(); - Path testFile = path("testOpenFileCost"); - - writeTextFile(fs, testFile, "openfile", true); - FileStatus st = fs.getFileStatus(testFile); // now read that file back in using the openFile call. // with a new FileStatus and a different path. // this verifies that any FileStatus class/subclass is used // as a source of the file length. - long len = st.getLen(); FileStatus st2 = new FileStatus( - len, false, - st.getReplication(), - st.getBlockSize(), - st.getModificationTime(), - st.getAccessTime(), - st.getPermission(), - st.getOwner(), - st.getGroup(), + fileLength, false, + testFileStatus.getReplication(), + testFileStatus.getBlockSize(), + testFileStatus.getModificationTime(), + testFileStatus.getAccessTime(), + testFileStatus.getPermission(), + testFileStatus.getOwner(), + testFileStatus.getGroup(), new Path("gopher:///localhost/" + testFile.getName())); - // no IO + // no IO in open FSDataInputStream in = verifyMetrics(() -> fs.openFile(testFile) - .withFileStatus(st) + .withFileStatus(st2) .build() .get(), always(NO_IO), with(STREAM_OPENED, 0)); + // the stream gets opened during read long readLen = verifyMetrics(() -> readStream(in), always(NO_IO), with(STREAM_OPENED, 1)); + assertEquals("bytes read from file", fileLength, readLen); + } - - assertEquals("bytes read from file", len, readLen); - + @Test + public void testOpenFileShorterLength() throws Throwable { // do a second read with the length declared as short. // we now expect the bytes read to be shorter. + S3AFileSystem fs = getFileSystem(); + S3ATestUtils.MetricDiff bytesDiscarded = new S3ATestUtils.MetricDiff(fs, STREAM_CLOSE_BYTES_READ); int offset = 2; - long shortLen = len - offset; - CompletableFuture f2 = fs.openFile(testFile) - .must(FS_OPTION_OPENFILE_FADVISE, FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) - .opt(FS_OPTION_OPENFILE_LENGTH, shortLen) - .build(); + long shortLen = fileLength - offset; + // open the file FSDataInputStream in2 = verifyMetrics(() -> fs.openFile(testFile) .must(FS_OPTION_OPENFILE_FADVISE, @@ -143,6 +159,7 @@ public void testOpenFileCost() throws Throwable { always(NO_IO), with(STREAM_OPENED, 0)); + // now read it long r2 = verifyMetrics(() -> readStream(in2), always(NO_IO), @@ -153,8 +170,15 @@ public void testOpenFileCost() throws Throwable { assertEquals("bytes read from file", shortLen, r2); // the read has been ranged bytesDiscarded.assertDiffEquals(0); + } - long longLen = len + 10; + @Test + public void testOpenFileLongerLength() throws Throwable { + // do a second read with the length declared as short. + // we now expect the bytes read to be shorter. + S3AFileSystem fs = getFileSystem(); + // now set a length past the actual file length + long longLen = fileLength + 10; FSDataInputStream in3 = verifyMetrics(() -> fs.openFile(testFile) .must(FS_OPTION_OPENFILE_FADVISE, @@ -164,7 +188,8 @@ public void testOpenFileCost() throws Throwable { .get(), always(NO_IO)); - // shows that irrespective of the declared length, you can read past it. + // shows that irrespective of the declared length, you can read past it, + // and an EOFException is raised verifyMetrics(() -> { byte[] out = new byte[(int) longLen]; intercept(EOFException.class, @@ -173,7 +198,7 @@ public void testOpenFileCost() throws Throwable { assertEquals("read past real EOF on " + in3, -1, in3.read()); in3.close(); - return null; + return in3.toString(); }, // two GET calls were made, one for readFully, // the second on the read() past the EOF diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java index 9a72c9d7dbfde..a83e59b8e3c76 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java @@ -52,6 +52,9 @@ import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; import static org.apache.hadoop.tools.mapred.CopyMapper.getFileAttributeSettings; /** @@ -362,7 +365,10 @@ private static ThrottledInputStream getInputStream(Path path, FileSystem fs = path.getFileSystem(conf); float bandwidthMB = conf.getFloat(DistCpConstants.CONF_LABEL_BANDWIDTH_MB, DistCpConstants.DEFAULT_BANDWIDTH_MB); - FSDataInputStream in = fs.open(path); + FSDataInputStream in = awaitFuture(fs.openFile(path) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .build()); return new ThrottledInputStream(in, bandwidthMB * 1024 * 1024); } catch (IOException e) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java index e5fb417561179..9435d0b6ed5e2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java @@ -60,7 +60,11 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; import org.apache.hadoop.yarn.exceptions.YarnException; -/** +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; + + /** * Download a single URL to the local disk. * */ @@ -285,23 +289,25 @@ private void verifyAndCopy(Path destination) } } - downloadAndUnpack(sCopy, destination); + downloadAndUnpack(sCopy, sStat, destination); } /** * Copy source path to destination with localization rules. - * @param source source path to copy. Typically HDFS + * @param source source path to copy. Typically HDFS or an object store. + * @param sourceStatus status of source * @param destination destination path. Typically local filesystem * @exception YarnException Any error has occurred */ - private void downloadAndUnpack(Path source, Path destination) + private void downloadAndUnpack(Path source, + FileStatus sourceStatus, Path destination) throws YarnException { try { FileSystem sourceFileSystem = source.getFileSystem(conf); FileSystem destinationFileSystem = destination.getFileSystem(conf); - if (sourceFileSystem.getFileStatus(source).isDirectory()) { + if (sourceStatus.isDirectory()) { FileUtil.copy( - sourceFileSystem, source, + sourceFileSystem, sourceStatus, destinationFileSystem, destination, false, true, conf); } else { @@ -329,7 +335,11 @@ private void unpack(Path source, Path destination, FileSystem sourceFileSystem, FileSystem destinationFileSystem) throws IOException, InterruptedException, ExecutionException { - try (InputStream inputStream = sourceFileSystem.open(source)) { + try (InputStream inputStream = awaitFuture( + sourceFileSystem.openFile(source) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .build())) { File dst = new File(destination.toUri()); String lowerDst = StringUtils.toLowerCase(dst.getName()); switch (resource.getType()) { From 7d0bd3a7b94641d3d60505b3c0788e32415084ac Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 23 Sep 2020 19:07:52 +0100 Subject: [PATCH 05/10] HADOOP-16202. default must keys wired up; checkstyles Change-Id: I08045aaf13919f3923a7411ef64096f5e3171e69 --- .../java/org/apache/hadoop/fs/FileSystem.java | 2 +- .../fs/s3a/performance/ITestS3AOpenCost.java | 18 +++++++++--------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index 4a6729ed56dd9..bf0f7f3ff9ab4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -4627,7 +4627,7 @@ protected CompletableFuture openFileWithOptions( final OpenFileParameters parameters) throws IOException { AbstractFSBuilderImpl.rejectUnknownMandatoryKeys( parameters.getMandatoryKeys(), - Collections.emptySet(), + Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS, "for " + path); return LambdaUtils.eval( new CompletableFuture<>(), () -> diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java index 10703e0c44004..85d7fc0003149 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java @@ -191,15 +191,15 @@ public void testOpenFileLongerLength() throws Throwable { // shows that irrespective of the declared length, you can read past it, // and an EOFException is raised verifyMetrics(() -> { - byte[] out = new byte[(int) longLen]; - intercept(EOFException.class, - () -> in3.readFully(0, out)); - in3.seek(longLen - 1); - assertEquals("read past real EOF on " + in3, - -1, in3.read()); - in3.close(); - return in3.toString(); - }, + byte[] out = new byte[(int) longLen]; + intercept(EOFException.class, + () -> in3.readFully(0, out)); + in3.seek(longLen - 1); + assertEquals("read past real EOF on " + in3, + -1, in3.read()); + in3.close(); + return in3.toString(); + }, // two GET calls were made, one for readFully, // the second on the read() past the EOF // the operation has got as far as S3 From 66243dc1c8e3f7021ebf8d9708b3403305b763c8 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 22 Oct 2020 13:54:34 +0100 Subject: [PATCH 06/10] HADOOP-16202. buffer size is also supported in openFile * fs.option.buffer.size sets buffer size * this is picked up in the default FS/FC file openings * fadvise declared to take a CSV list of seek options, read in order until one is recognized by the store. This is to allow for custom policies and evolution over time. * review of FileContext.open and switch to openFile where sequential access likely to make a significant different (i.e. large files) * JsonSerialization to ignore available(), as that is just buffer capacity. * JsonSerialization exposes reader taking an FSDataInputStream. That's to let me switch the committers to openFile() with length -the committers should set the example here. It should significantly benefit Magic committer by eliminating 1 HEAD/.pending in task commit, one HEAD per .pendingset during job commit...overall 50% of the HEAD/GET I/O to S3 in these operations. Docs improved. Other than moving the magic committer to that new read (to be followup), this is done. Change-Id: I013d338f2b043f7940d7c17fcb2a7777cc7c1c18 --- .../org/apache/hadoop/fs/AvroFSInput.java | 13 +- .../org/apache/hadoop/fs/FileContext.java | 15 +- .../java/org/apache/hadoop/fs/FileSystem.java | 6 +- .../java/org/apache/hadoop/fs/Options.java | 12 +- .../org/apache/hadoop/io/SequenceFile.java | 14 +- .../apache/hadoop/util/JsonSerialization.java | 27 +++- .../src/site/markdown/filesystem/openfile.md | 17 ++- .../fs/contract/AbstractContractOpenTest.java | 15 +- .../jobhistory/JobHistoryCopyService.java | 11 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 11 +- .../apache/hadoop/fs/s3a/S3AInputPolicy.java | 42 ++++-- .../hadoop/fs/s3a/impl/InternalConstants.java | 22 +-- .../fs/s3a/impl/S3AOpenFileOperation.java | 99 ++++++++----- .../fs/contract/s3a/ITestS3AContractSeek.java | 5 +- .../fs/s3a/impl/TestOpenFileHelper.java | 140 +++++++++++++++--- .../logaggregation/AggregatedLogFormat.java | 17 ++- 16 files changed, 365 insertions(+), 101 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java index b4a4a85674dfa..d23a499ccff7d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java @@ -25,6 +25,11 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; + /** Adapts an {@link FSDataInputStream} to Avro's SeekableInput interface. */ @InterfaceAudience.Public @InterfaceStability.Stable @@ -42,7 +47,13 @@ public AvroFSInput(final FSDataInputStream in, final long len) { public AvroFSInput(final FileContext fc, final Path p) throws IOException { FileStatus status = fc.getFileStatus(p); this.len = status.getLen(); - this.stream = fc.open(p); + this.stream = awaitFuture(fc.openFile(p) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .opt(FS_OPTION_OPENFILE_LENGTH, + status.getLen()) // file length hint for object stores + .build()); + fc.open(p); } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java index 32824b4824831..28e297ed29c83 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java @@ -71,6 +71,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; /** @@ -2204,7 +2209,12 @@ public boolean copy(final Path src, final Path dst, boolean deleteSource, EnumSet createFlag = overwrite ? EnumSet.of( CreateFlag.CREATE, CreateFlag.OVERWRITE) : EnumSet.of(CreateFlag.CREATE); - InputStream in = open(qSrc); + InputStream in = awaitFuture(openFile(qSrc) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .opt(FS_OPTION_OPENFILE_LENGTH, + fs.getLen()) // file length hint for object stores + .build()); try (OutputStream out = create(qDst, createFlag)) { IOUtils.copyBytes(in, out, conf, true); } finally { @@ -2940,6 +2950,9 @@ public CompletableFuture build() throws IOException { .withOptions(getOptions()) .withBufferSize(getBufferSize()) .withStatus(getStatus()); + parameters.withBufferSize( + getOptions().getInt(FS_OPTION_OPENFILE_BUFFER_SIZE, + getBufferSize())); return new FSLinkResolver>() { @Override public CompletableFuture next( diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index bf0f7f3ff9ab4..2c32c6af66aca 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -91,6 +91,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; @@ -4764,8 +4765,11 @@ public CompletableFuture build() throws IOException { .withMandatoryKeys(getMandatoryKeys()) .withOptionalKeys(getOptionalKeys()) .withOptions(getOptions()) - .withBufferSize(getBufferSize()) .withStatus(super.getStatus()); // explicit to avoid IDE warnings + // buffer size can be configured + parameters.withBufferSize( + getOptions().getInt(FS_OPTION_OPENFILE_BUFFER_SIZE, + getBufferSize())); if(optionalPath.isPresent()) { return getFS().openFileWithOptions(optionalPath.get(), parameters); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java index 9d8be90a30d62..a9e7bcd90676b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.fs; +import java.util.Collections; import java.util.Optional; import java.util.Set; import java.util.function.Function; @@ -549,6 +550,12 @@ private OpenFileOptions() { public static final String FS_OPTION_OPENFILE_LENGTH = FS_OPTION_OPENFILE + "length"; + /** + * 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}. */ @@ -584,10 +591,11 @@ private OpenFileOptions() { * MUST recognize, even if they ignore the actual values. */ public static final Set FS_OPTION_OPENFILE_STANDARD_OPTIONS = - Stream.of( + Collections.unmodifiableSet(Stream.of( + FS_OPTION_OPENFILE_BUFFER_SIZE, FS_OPTION_OPENFILE_FADVISE, FS_OPTION_OPENFILE_LENGTH) - .collect(Collectors.toSet()); + .collect(Collectors.toSet())); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java index 3f4649f04dc9a..078805abbd52f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java @@ -59,6 +59,11 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SEQFILE_COMPRESS_BLOCKSIZE_KEY; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SKIP_CHECKSUM_ERRORS_DEFAULT; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SKIP_CHECKSUM_ERRORS_KEY; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; /** * SequenceFiles are flat files consisting of binary key/value @@ -1942,7 +1947,14 @@ private void initialize(Path filename, FSDataInputStream in, */ protected FSDataInputStream openFile(FileSystem fs, Path file, int bufferSize, long length) throws IOException { - return fs.open(file, bufferSize); + FutureDataInputStreamBuilder builder = fs.openFile(file) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .opt(FS_OPTION_OPENFILE_BUFFER_SIZE, bufferSize); + if (length >= 0) { + builder.opt(FS_OPTION_OPENFILE_LENGTH, length); + } + return awaitFuture(builder.build()); } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java index 4b4ea6b415d10..f6a9c753a993b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java @@ -35,6 +35,9 @@ import com.fasterxml.jackson.databind.ObjectReader; import com.fasterxml.jackson.databind.ObjectWriter; import com.fasterxml.jackson.databind.SerializationFeature; + +import org.apache.hadoop.fs.FutureDataInputStreamBuilder; +import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; @@ -257,11 +260,29 @@ public T load(FileSystem fs, Path path) throws IOException { if (dataInputStream.available() == 0) { throw new EOFException("No data in " + path); } + return fromDataInputStream(path, dataInputStream); + } + } + + /** + * Read from a data input stream; convert all exceptions to + * IOExceptions. + * The stream is not explicitly closed. + * @param path path for errors + * @param dataInputStream source of data + * @return a loaded object + * @throws PathIOException IO or JSON parse problems + */ + public T fromDataInputStream(final Path path, + final FSDataInputStream dataInputStream) + throws IOException { + // throw an EOF exception if there is no data available. + try { return fromJsonStream(dataInputStream); } catch (JsonProcessingException e) { - throw new IOException( - String.format("Failed to read JSON file \"%s\": %s", path, e), - e); + throw (IOException) new PathIOException(path.toString(), + "Failed to read JSON file " + e) + .initCause(e); } } diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md index df51d043b45bb..353b28d5a18d6 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md @@ -138,7 +138,17 @@ 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. -They MUST be viewed as hints. +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` @@ -153,9 +163,14 @@ data is more efficient -so may be used instead. | `normal` | The "Normal" policy for this store. | | `sequential` | Optimized for sequential access. | | `random` | Optimized purely for random seek+read/positionedRead operations. | +| `adaptive` | Any adaptive policy implemented by the store. | Choosing the wrong read policy for an input source may be inefficient. +A list of seek policies MAY be supplied; the first one recognized by the filesystem +SHALL be the one used. This allows for custom policies to be supported, for example +an `hbase-hfile` policy optimized for HBase HFiles. + Tip: log the `toString()` value of input streams at `DEBUG`. The S3A and ABFS Input Streams log read statistics, which can provide insight about whether reads are being performed efficiently or not. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java index cf6dc5571ec55..c3eb8adbc7c23 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java @@ -33,7 +33,9 @@ import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.io.IOUtils; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_RANDOM; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.contract.ContractTestUtils.compareByteArrays; @@ -341,7 +343,7 @@ public void testOpenFileApplyAsyncRead() throws Throwable { @Test public void testOpenFileNullStatusButFileLength() throws Throwable { describe("use openFile() with a null status and expect the status to be" - + " ignored. fadvise and length are passed in as must() options"); + + " ignored. block size, fadvise and length are passed in as must() options"); Path path = path("testOpenFileNullStatus"); FileSystem fs = getFileSystem(); int len = 4; @@ -352,13 +354,16 @@ public void testOpenFileNullStatusButFileLength() throws Throwable { CompletableFuture future = fs.openFile(path) .withFileStatus(null) .must(FS_OPTION_OPENFILE_FADVISE, - FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) - .must(FS_OPTION_OPENFILE_LENGTH, - len) + "unknown, " + + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL + + ", " + + FS_OPTION_OPENFILE_FADVISE_RANDOM) + .must(FS_OPTION_OPENFILE_BUFFER_SIZE, 32768) + .must(FS_OPTION_OPENFILE_LENGTH, len) .build(); try (FSDataInputStream in = future.get()) { - in.read(result, 0, len); + in.readFully(result); } compareByteArrays(dataset, result, len); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java index ee4ec2c86a1c4..792e1c2a124d3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java @@ -35,6 +35,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; + /** * Reads in history events from the JobHistoryFile and sends them out again * to be recorded. @@ -118,7 +123,11 @@ public static FSDataInputStream getPreviousJobHistoryFileStream( fc.makeQualified(JobHistoryUtils.getStagingJobHistoryFile(histDirPath, jobId, (applicationAttemptId.getAttemptId() - 1))); LOG.info("History file is at " + historyFile); - in = fc.open(historyFile); + in = awaitFuture( + fc.openFile(historyFile) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .build()); return in; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index b27e614c538cf..4c45027f84f73 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -168,6 +168,8 @@ import org.apache.hadoop.util.SemaphoredDelegatingExecutor; import org.apache.hadoop.util.concurrent.HadoopExecutors; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Invoker.*; @@ -424,7 +426,8 @@ public void initialize(URI name, Configuration originalConf) doBucketProbing(); inputPolicy = S3AInputPolicy.getPolicy( - conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL)); + conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL), + S3AInputPolicy.Normal); LOG.debug("Input fadvise policy = {}", inputPolicy); changeDetectionPolicy = ChangeDetectionPolicy.getPolicy(conf); LOG.debug("Change detection policy = {}", changeDetectionPolicy); @@ -488,7 +491,9 @@ public void initialize(URI name, Configuration originalConf) inputPolicy, changeDetectionPolicy, readAhead, - username); + username, + intOption(getConf(), IO_FILE_BUFFER_SIZE_KEY, + IO_FILE_BUFFER_SIZE_DEFAULT, 0)); } catch (AmazonClientException e) { // amazon client exception: stop all services then throw the translation stopAllServices(); @@ -1112,7 +1117,7 @@ protected URI canonicalizeUri(URI rawUri) { public FSDataInputStream open(Path f, int bufferSize) throws IOException { entryPoint(INVOCATION_OPEN); - return open(qualify(f), openFileHelper.openSimpleFile()); + return open(qualify(f), openFileHelper.openSimpleFile(bufferSize)); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java index eb3539dd076c9..d41f0fa9786a4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputPolicy.java @@ -18,15 +18,17 @@ package org.apache.hadoop.fs.s3a; +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.Locale; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Locale; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_ADAPTIVE; -import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADV_NORMAL; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADV_RANDOM; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADV_SEQUENTIAL; /** * Filesystem input policy. @@ -39,8 +41,6 @@ public enum S3AInputPolicy { Sequential(INPUT_FADV_SEQUENTIAL), Random(INPUT_FADV_RANDOM); - private static final Logger LOG = - LoggerFactory.getLogger(S3AInputPolicy.class); private final String policy; S3AInputPolicy(String policy) { @@ -54,12 +54,13 @@ public String toString() { /** * Choose an FS access policy. - * Always returns something, - * primarily by downgrading to "normal" if there is no other match. * @param name strategy name from a configuration option, etc. + * @param defaultPolicy default policy to fall back to. * @return the chosen strategy */ - public static S3AInputPolicy getPolicy(String name) { + public static S3AInputPolicy getPolicy( + String name, + @Nullable S3AInputPolicy defaultPolicy) { String trimmed = name.trim().toLowerCase(Locale.ENGLISH); switch (trimmed) { case INPUT_FADV_NORMAL: @@ -70,9 +71,26 @@ public static S3AInputPolicy getPolicy(String name) { case INPUT_FADV_SEQUENTIAL: return Sequential; default: - LOG.warn("Unrecognized " + INPUT_FADVISE + " value: \"{}\"", trimmed); - return Normal; + return defaultPolicy; + } + } + + /** + * Scan the list of input policies, returning the first one supported. + * @param policies list of policies. + * @param defaultPolicy fallback + * @return a policy or the defaultPolicy, which may be null + */ + public static S3AInputPolicy getFirstSupportedPolicy( + Collection policies, + @Nullable S3AInputPolicy defaultPolicy) { + for (String s : policies) { + S3AInputPolicy nextPolicy = S3AInputPolicy.getPolicy(s, null); + if (nextPolicy != null) { + return nextPolicy; + } } + return defaultPolicy; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 73d13ca7e77b8..e0413cc75ea8c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.impl; +import java.util.Collections; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -27,8 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.s3a.Constants; -import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; -import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS; /** * Internal constants private only to the S3A codebase. @@ -81,14 +81,16 @@ private InternalConstants() { * used becomes that of the select operation. */ @InterfaceStability.Unstable - public static final Set S3A_OPENFILE_KEYS = - Stream.of( - FS_OPTION_OPENFILE_FADVISE, - FS_OPTION_OPENFILE_LENGTH, - Constants.INPUT_FADVISE, - Constants.READAHEAD_RANGE) - .collect(Collectors.toSet()); - + public static final Set S3A_OPENFILE_KEYS; + + static { + Set keys = Stream.of( + Constants.INPUT_FADVISE, + Constants.READAHEAD_RANGE) + .collect(Collectors.toSet()); + keys.addAll(FS_OPTION_OPENFILE_STANDARD_OPTIONS); + S3A_OPENFILE_KEYS = Collections.unmodifiableSet(keys); + } /** 404 error code. */ public static final int SC_404 = 404; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java index 89b87582b35c4..d4078dfedf425 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java @@ -20,6 +20,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.util.Collection; import java.util.Set; import org.slf4j.Logger; @@ -35,62 +36,70 @@ import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; import org.apache.hadoop.fs.s3a.select.SelectConstants; -import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; +import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; /** * Helper class for openFile() logic, especially processing file status * args and length/etag/versionID. - *

- * This got complex enough it merited removal from S3AFS -which - * also permits unit testing. + *

+ * This got complex enough it merited removal from S3AFileSystemy -which + * also permits unit testing. + *

+ *

+ * The default values are those from the FileSystem configuration. + * in openFile(), they can all be changed by specific options; + * in FileSystem.open(path, buffersize) only the buffer size is + * set. + *

*/ public class S3AOpenFileOperation extends AbstractStoreOperation { private static final Logger LOG = LoggerFactory.getLogger(S3AOpenFileOperation.class); - private final S3AInputPolicy inputPolicy; + /** Default input policy. */ + private final S3AInputPolicy defaultInputPolicy; + /** Default change detection policy. */ private final ChangeDetectionPolicy changePolicy; - private final long readAheadRange; + /** Default read ahead range. */ + private final long defaultReadAhead; + /** Username. */ private final String username; - /** - * Simple file information is created in advance as it is always - * the same. - */ - private final OpenFileInformation simpleFileInformation; + /** Default buffer size. */ + private final int defaultBufferSize; /** * Instantiate with the default options from the filesystem. - * @param inputPolicy input policy + * @param defaultInputPolicy input policy * @param changePolicy change detection policy - * @param readAheadRange read ahead range + * @param defaultReadAhead read ahead range * @param username username + * @param defaultBufferSize buffer size */ public S3AOpenFileOperation( - final S3AInputPolicy inputPolicy, + final S3AInputPolicy defaultInputPolicy, final ChangeDetectionPolicy changePolicy, - final long readAheadRange, - final String username) { + final long defaultReadAhead, + final String username, + final int defaultBufferSize) { super(null); - this.inputPolicy = inputPolicy; + this.defaultInputPolicy = defaultInputPolicy; this.changePolicy = changePolicy; - this.readAheadRange = readAheadRange; + this.defaultReadAhead = defaultReadAhead; this.username = username; - - simpleFileInformation = new OpenFileInformation(false, null, null, - inputPolicy, changePolicy, readAheadRange); + this.defaultBufferSize = defaultBufferSize; } - /** * The information on a file needed to open it. */ @@ -114,6 +123,9 @@ public static final class OpenFileInformation { /** read ahead range. */ private final long readAheadRange; + /** Buffer size. Currently ignored. */ + private final int bufferSize; + /** * Constructor. */ @@ -122,13 +134,15 @@ private OpenFileInformation(final boolean isSql, final String sql, final S3AInputPolicy inputPolicy, final ChangeDetectionPolicy changePolicy, - final long readAheadRange) { + final long readAheadRange, + final int bufferSize) { this.isSql = isSql; this.status = status; this.sql = sql; this.inputPolicy = inputPolicy; this.changePolicy = changePolicy; this.readAheadRange = readAheadRange; + this.bufferSize = bufferSize; } public boolean isSql() { @@ -155,6 +169,10 @@ public long getReadAheadRange() { return readAheadRange; } + public int getBufferSize() { + return bufferSize; + } + @Override public String toString() { return "OpenFileInformation{" + @@ -164,6 +182,7 @@ public String toString() { ", inputPolicy=" + inputPolicy + ", changePolicy=" + changePolicy + ", readAheadRange=" + readAheadRange + + ", bufferSize=" + bufferSize + '}'; } } @@ -224,7 +243,6 @@ public OpenFileInformation prepareToOpenFile( // can use this status to skip our own probes, LOG.debug("File was opened with a supplied FileStatus;" + " skipping getFileStatus call in open() operation: {}", - providedStatus); if (providedStatus instanceof S3AFileStatus) { S3AFileStatus st = (S3AFileStatus) providedStatus; @@ -269,23 +287,36 @@ public OpenFileInformation prepareToOpenFile( // neither a file status nor a file length fileStatus = null; } - // seek policy from default, s3a opt or standard option - String policy1 = options.get(INPUT_FADVISE, inputPolicy.toString()); - String policy2 = options.get(FS_OPTION_OPENFILE_FADVISE, policy1); - S3AInputPolicy policy = S3AInputPolicy.getPolicy(policy2); + // Build up the input policy. + // seek policy from default, s3a opt or standard option + // read from the FS standard option. + Collection policies = + options.getStringCollection(FS_OPTION_OPENFILE_FADVISE); + if (policies.isEmpty()) { + // fall back to looking at the S3A-specific option. + policies = options.getStringCollection(INPUT_FADVISE); + } + // get the first known policy + S3AInputPolicy seekPolicy = S3AInputPolicy.getFirstSupportedPolicy(policies, + defaultInputPolicy); // readahead range - long ra = options.getLong(READAHEAD_RANGE, readAheadRange); - return new OpenFileInformation(isSelect, fileStatus, sql, policy, - changePolicy, ra); + long readAhead = options.getLong(READAHEAD_RANGE, defaultReadAhead); + // buffer size + int bufferSize = options.getInt(FS_OPTION_OPENFILE_BUFFER_SIZE, + defaultBufferSize); + return new OpenFileInformation(isSelect, fileStatus, sql, seekPolicy, + changePolicy, readAhead, bufferSize); } /** * Open a simple file. - * @return the parameters needed to open a file through open(). + * @return the parameters needed to open a file through open(path, bufferSize). + * @param bufferSize buffer size */ - public OpenFileInformation openSimpleFile() { - return simpleFileInformation; + public OpenFileInformation openSimpleFile(final int bufferSize) { + return new OpenFileInformation(false, null, null, + defaultInputPolicy, changePolicy, defaultReadAhead, bufferSize); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java index bc8b71b3b3bf4..e906fc2b23d24 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java @@ -24,6 +24,7 @@ import java.util.Arrays; import java.util.Collection; +import org.assertj.core.api.Assertions; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -44,6 +45,7 @@ import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.NativeCodeLoader; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADV_NORMAL; @@ -217,7 +219,8 @@ public void validateSSLChannelMode() { public void testReadPolicyInFS() throws Throwable { describe("Verify the read policy is being consistently set"); S3AFileSystem fs = getFileSystem(); - assertEquals(S3AInputPolicy.getPolicy(seekPolicy), fs.getInputPolicy()); + assertEquals(S3AInputPolicy.getPolicy(seekPolicy, S3AInputPolicy.Normal), + fs.getInputPolicy()); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java index 494bb70e9e2b7..cff4fcde9fdd2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java @@ -20,10 +20,10 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.util.Collection; import org.assertj.core.api.Assertions; import org.assertj.core.api.ObjectAssert; -import org.junit.Before; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -35,15 +35,21 @@ import org.apache.hadoop.test.HadoopTestBase; import static java.util.Collections.singleton; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_ADAPTIVE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_NORMAL; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_RANDOM; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** - * Unit tests for {@link S3AOpenFileOperation}. + * Unit tests for {@link S3AOpenFileOperation} and the associated + * seek policy lookup in {@link S3AInputPolicy}. */ public class TestOpenFileHelper extends HadoopTestBase { @@ -61,18 +67,18 @@ public class TestOpenFileHelper extends HadoopTestBase { private static final Path TESTPATH = new Path(TESTFILE); - private S3AOpenFileOperation helper; - - @Before - public void setup() { - helper = new S3AOpenFileOperation(INPUT_POLICY, - CHANGE_POLICY, READ_AHEAD_RANGE, USERNAME); - } + private static final S3AOpenFileOperation OPERATION = + new S3AOpenFileOperation( + INPUT_POLICY, + CHANGE_POLICY, + READ_AHEAD_RANGE, + USERNAME, + IO_FILE_BUFFER_SIZE_DEFAULT); @Test public void testSimpleFile() throws Throwable { ObjectAssert - asst = assertFI(helper.openSimpleFile()); + asst = assertFI(OPERATION.openSimpleFile(1024)); asst.extracting(f -> f.getChangePolicy()) .isEqualTo(CHANGE_POLICY); @@ -93,6 +99,13 @@ private ObjectAssert assertFI( .describedAs("File Information %s", fi); } + /** + * Create an assertion about the openFile information from a configuration + * with the given key/value option. + * @param key key to set. + * @param option option value. + * @return the constructed OpenFileInformation. + */ public ObjectAssert assertOpenFile( final String key, final String option) throws IOException { return assertFI(prepareToOpenFile(params(key, option))); @@ -113,22 +126,28 @@ public void testSeekRandomIOPolicy() throws Throwable { String option = FS_OPTION_OPENFILE_FADVISE_RANDOM; // is picked up - assertOpenFile(INPUT_FADVISE, option).extracting(f -> f.getInputPolicy()) + assertOpenFile(INPUT_FADVISE, option) + .extracting(f -> f.getInputPolicy()) .isEqualTo(S3AInputPolicy.Random); // and as neither status nor length was set: no file status - assertOpenFile(INPUT_FADVISE, option).extracting(f -> f.getStatus()) + assertOpenFile(INPUT_FADVISE, option) + .extracting(f -> f.getStatus()) .isNull(); } + /** + * There's a standard policy name. 'adaptive', + * meaning 'whatever this stream does to adapt to the client's use'. + * On the S3A connector that is mapped to Normal. + */ @Test public void testSeekPolicyAdaptive() throws Throwable { // when caller asks for adaptive, they get "normal" - ObjectAssert asst = - assertOpenFile(INPUT_FADVISE, FS_OPTION_OPENFILE_FADVISE_ADAPTIVE); - - asst.extracting(f -> f.getInputPolicy()) + assertOpenFile(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_ADAPTIVE) + .extracting(f -> f.getInputPolicy()) .isEqualTo(S3AInputPolicy.Normal); } @@ -136,28 +155,96 @@ public void testSeekPolicyAdaptive() throws Throwable { * Verify that an unknown seek policy falls back to "normal". */ @Test - public void testUnknownSeekPolicy() throws Throwable { - + public void testUnknownSeekPolicyS3AOption() throws Throwable { // fall back to the normal seek policy. - assertOpenFile(INPUT_FADVISE, "undefined") .extracting(f -> f.getInputPolicy()) + .isEqualTo(INPUT_POLICY); + } + + /** + * The S3A option also supports a list of values. + */ + @Test + public void testSeekPolicyListS3AOption() throws Throwable { + // fall back to the normal seek policy. + assertOpenFile(INPUT_FADVISE, "hbase, random") + .extracting(f -> f.getInputPolicy()) + .isEqualTo(S3AInputPolicy.Random); + } + + /** + * Verify that if a list of policies is supplied in a configuration, + * the first recognized policy will be adopted. + */ + @Test + public void testSeekPolicyExtractionFromList() throws Throwable { + String plist = "a, b, RandOm, other "; + Configuration conf = conf(FS_OPTION_OPENFILE_FADVISE, plist); + Collection options = conf.getTrimmedStringCollection( + FS_OPTION_OPENFILE_FADVISE); + Assertions.assertThat(S3AInputPolicy.getFirstSupportedPolicy(options, null)) + .describedAs("Policy from " + plist) + .isEqualTo(S3AInputPolicy.Random); + } + + @Test + public void testAdaptiveSeekPolicyRecognized() throws Throwable { + Assertions.assertThat(S3AInputPolicy.getPolicy("adaptive", null)) + .describedAs("adaptive") .isEqualTo(S3AInputPolicy.Normal); } + @Test + public void testUnknownSeekPolicyFallback() throws Throwable { + Assertions.assertThat(S3AInputPolicy.getPolicy("unknown", null)) + .describedAs("unkown policy") + .isNull(); + } + + /** + * Test the mapping of the standard option names. + */ + @Test + public void testWellKnownPolicyMapping() throws Throwable { + Object[][] policyMapping = { + {"normal", S3AInputPolicy.Normal}, + {FS_OPTION_OPENFILE_FADVISE_NORMAL, S3AInputPolicy.Normal}, + {FS_OPTION_OPENFILE_FADVISE_ADAPTIVE, S3AInputPolicy.Normal}, + {FS_OPTION_OPENFILE_FADVISE_RANDOM, S3AInputPolicy.Random}, + {FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL, S3AInputPolicy.Sequential}, + }; + for (Object[] mapping : policyMapping) { + String name = (String) mapping[0]; + Assertions.assertThat(S3AInputPolicy.getPolicy(name, null)) + .describedAs("Policy %s", name) + .isEqualTo(mapping[1]); + } + } + + /** * Verify readahead range is picked up. */ @Test public void testReadahead() throws Throwable { - // readahead range option - assertOpenFile(READAHEAD_RANGE, "4096") .extracting(f -> f.getReadAheadRange()) .isEqualTo(4096L); } + /** + * Verify readahead range is picked up. + */ + @Test + public void testBufferSize() throws Throwable { + // readahead range option + assertOpenFile(FS_OPTION_OPENFILE_BUFFER_SIZE, "4096") + .extracting(f -> f.getBufferSize()) + .isEqualTo(4096); + } + @Test public void testStatusWithValidFilename() throws Throwable { Path p = new Path("file:///tmp/" + TESTPATH.getName()); @@ -216,11 +303,17 @@ public void testStatusWithInconsistentFilename() throws Throwable { new Path(TESTFILE + "-"), USERNAME)))); } + /** + * Prepare to open a file with the set of parameters. + * @param parameters open a file + * @return + * @throws IOException + */ public S3AOpenFileOperation.OpenFileInformation prepareToOpenFile( final OpenFileParameters parameters) throws IOException { - return helper.prepareToOpenFile(TESTPATH, - parameters, 0); + return OPERATION.prepareToOpenFile(TESTPATH, + parameters, IO_FILE_BUFFER_SIZE_DEFAULT); } /** @@ -277,4 +370,5 @@ private Configuration conf(String key, Object val) { c.set(key, val.toString()); return c; } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java index 0fa9764b7bb1f..4859ef82202f8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java @@ -56,6 +56,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; @@ -77,6 +78,11 @@ import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables; import org.apache.hadoop.thirdparty.com.google.common.collect.Sets; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; + @Public @Evolving public class AggregatedLogFormat { @@ -576,9 +582,16 @@ public LogReader(Configuration conf, Path remoteAppLogFile) try { FileContext fileContext = FileContext.getFileContext(remoteAppLogFile.toUri(), conf); - this.fsDataIStream = fileContext.open(remoteAppLogFile); + FileStatus status = fileContext.getFileStatus(remoteAppLogFile); + this.fsDataIStream = awaitFuture( + fileContext.openFile(remoteAppLogFile) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .opt(FS_OPTION_OPENFILE_LENGTH, + status.getLen()) // file length hint for object stores + .build()); reader = new TFile.Reader(this.fsDataIStream, - fileContext.getFileStatus(remoteAppLogFile).getLen(), conf); + status.getLen(), conf); this.scanner = reader.createScanner(); } catch (IOException ioe) { close(); From f56ea291054301415651a2b152c50e524eeaf466 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 17 Nov 2020 21:32:56 +0000 Subject: [PATCH 07/10] HADOOP-16202: JsonSerialization optimisation; S3A support tuning JsonSerialization takes an optional filestatus now. Exposed protected constructor to PathIOException to help there. Avoids playing initCause games. Test failures in ITestS3AInputStreamPerformance switched me to passing down the (changeable) seek policy in the FS; the fil aree opening there all switches to the openFile() API with the standard names, so verifies in an ITest that they are being picked up correctly. Change-Id: Ief8c5d40c3e1ca829afcabb27e9e04aecad8bb48 --- .../org/apache/hadoop/fs/PathIOException.java | 8 ++- .../apache/hadoop/util/JsonSerialization.java | 63 ++++++++----------- .../hadoop/util/TestJsonSerialization.java | 36 +++++++++++ .../org/apache/hadoop/fs/s3a/Constants.java | 10 ++- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 14 +++-- .../apache/hadoop/fs/s3a/S3AInputStream.java | 2 +- .../fs/s3a/impl/S3AOpenFileOperation.java | 24 +++---- .../fs/s3a/impl/TestOpenFileHelper.java | 7 ++- .../scale/ITestS3AInputStreamPerformance.java | 25 ++++---- 9 files changed, 117 insertions(+), 72 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PathIOException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PathIOException.java index deb3880ee4195..f32f2a93544bf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PathIOException.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PathIOException.java @@ -64,7 +64,13 @@ public PathIOException(String path, String error) { this.path = path; } - protected PathIOException(String path, String error, Throwable cause) { + /** + * Use a subclass of PathIOException if possible. + * @param path for the exception + * @param error custom string to use an the error text + * @param cause cause of exception. + */ + public PathIOException(String path, String error, Throwable cause) { super(error, cause); this.path = path; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java index f6a9c753a993b..76b35d6ccde44 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java @@ -36,8 +36,6 @@ import com.fasterxml.jackson.databind.ObjectWriter; import com.fasterxml.jackson.databind.SerializationFeature; -import org.apache.hadoop.fs.FutureDataInputStreamBuilder; -import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; @@ -46,8 +44,10 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; @@ -236,53 +236,44 @@ public T fromInstance(T instance) throws IOException { /** * Load from a Hadoop filesystem. - * There's a check for data availability after the file is open, by - * raising an EOFException if stream.available == 0. - * This allows for a meaningful exception without the round trip overhead - * of a getFileStatus call before opening the file. It may be brittle - * against an FS stream which doesn't return a value here, but the - * standard filesystems all do. - * JSON parsing and mapping problems - * are converted to IOEs. * @param fs filesystem * @param path path * @return a loaded object - * @throws IOException IO or JSON parse problems + * @throws PathIOException JSON parse problem + * @throws IOException IO problems */ public T load(FileSystem fs, Path path) throws IOException { - - try (FSDataInputStream dataInputStream = - awaitFuture(fs.openFile(path) - .opt(FS_OPTION_OPENFILE_FADVISE, - FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) - .build())) { - // throw an EOF exception if there is no data available. - if (dataInputStream.available() == 0) { - throw new EOFException("No data in " + path); - } - return fromDataInputStream(path, dataInputStream); - } + return load(fs, path, null); } /** - * Read from a data input stream; convert all exceptions to - * IOExceptions. - * The stream is not explicitly closed. - * @param path path for errors - * @param dataInputStream source of data + * Load from a Hadoop filesystem. + * If a file status is supplied, it's passed in to the openFile() + * call so that FS implementations can optimize their opening. + * @param fs filesystem + * @param path path + * @param status status of the file to open. * @return a loaded object - * @throws PathIOException IO or JSON parse problems + * @throws PathIOException JSON parse problem + * @throws EOFException file status references an empty file + * @throws IOException IO problems */ - public T fromDataInputStream(final Path path, - final FSDataInputStream dataInputStream) + public T load(FileSystem fs, Path path, FileStatus status) throws IOException { - // throw an EOF exception if there is no data available. - try { + + if (status != null && status.getLen() == 0) { + throw new EOFException("No data in " + path); + } + try (FSDataInputStream dataInputStream = + awaitFuture(fs.openFile(path) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL) + .withFileStatus(status) + .build())) { return fromJsonStream(dataInputStream); } catch (JsonProcessingException e) { - throw (IOException) new PathIOException(path.toString(), - "Failed to read JSON file " + e) - .initCause(e); + throw new PathIOException(path.toString(), + "Failed to read JSON file " + e, e); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJsonSerialization.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJsonSerialization.java index 991697d96bc95..fb0a81e55f1f3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJsonSerialization.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJsonSerialization.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import java.util.Objects; import com.fasterxml.jackson.core.JsonParseException; @@ -31,9 +32,13 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.test.HadoopTestBase; import org.apache.hadoop.test.LambdaTestUtils; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + /** * Test the JSON serialization helper. */ @@ -181,5 +186,36 @@ public void testFileSystemEmptyPath() throws Throwable { } } + @Test + public void testFileSystemWithStatus() throws Throwable { + File tempFile = File.createTempFile("Keyval", ".json"); + Path tempPath = new Path(tempFile.toURI()); + LocalFileSystem fs = FileSystem.getLocal(new Configuration()); + try { + serDeser.save(fs, tempPath, source, true); + KeyVal load = serDeser.load(fs, tempPath, fs.getFileStatus(tempPath)); + assertEquals("loaded values", source, load); + } finally { + fs.delete(tempPath, false); + } + } + + /** + * Handing a JSON parser an XML document must fail. + */ + @Test + public void testParseError() throws Throwable { + LocalFileSystem fs = FileSystem.getLocal(new Configuration()); + File tempFile = File.createTempFile("Keyval", ".json"); + Path tempPath = new Path(tempFile.toURI()); + try { + ContractTestUtils.createFile(fs, tempPath, true, + "".getBytes(StandardCharsets.UTF_8)); + intercept(PathIOException.class, () -> + serDeser.load(fs, tempPath, fs.getFileStatus(tempPath))); + } finally { + fs.delete(tempPath, false); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 3da528f625a5b..c9b66e39482d6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -20,6 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Options; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import java.util.concurrent.TimeUnit; @@ -529,13 +530,15 @@ private Constants() { * General input. Some seeks, some reads. * Value: {@value} */ - public static final String INPUT_FADV_NORMAL = "normal"; + public static final String INPUT_FADV_NORMAL = + Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_NORMAL; /** * Optimized for sequential access. * Value: {@value} */ - public static final String INPUT_FADV_SEQUENTIAL = "sequential"; + public static final String INPUT_FADV_SEQUENTIAL = + Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; /** * Optimized purely for random seek+read/positionedRead operations; @@ -543,7 +546,8 @@ private Constants() { * more efficient {@code seek()} operations. * Value: {@value} */ - public static final String INPUT_FADV_RANDOM = "random"; + public static final String INPUT_FADV_RANDOM = + Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_RANDOM; @InterfaceAudience.Private @InterfaceStability.Unstable diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 4c45027f84f73..001172a833462 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -488,7 +488,6 @@ public void initialize(URI name, Configuration originalConf) listing = new Listing(listingOperationCallbacks, createStoreContext()); // now the open file logic openFileHelper = new S3AOpenFileOperation( - inputPolicy, changeDetectionPolicy, readAhead, username, @@ -992,9 +991,11 @@ CannedAccessControlList getCannedACL() { /** * Change the input policy for this FS. + * @deprecated use openFile() options * @param inputPolicy new policy */ @InterfaceStability.Unstable + @Deprecated public void setInputPolicy(S3AInputPolicy inputPolicy) { Objects.requireNonNull(inputPolicy, "Null inputStrategy"); LOG.debug("Setting input strategy: {}", inputPolicy); @@ -1117,7 +1118,8 @@ protected URI canonicalizeUri(URI rawUri) { public FSDataInputStream open(Path f, int bufferSize) throws IOException { entryPoint(INVOCATION_OPEN); - return open(qualify(f), openFileHelper.openSimpleFile(bufferSize)); + return open(qualify(f), + openFileHelper.openSimpleFile(bufferSize, inputPolicy)); } /** @@ -1205,7 +1207,8 @@ private S3ObjectAttributes createObjectAttributes( * @return attributes to use when building the query. */ private S3ObjectAttributes createObjectAttributes( - final Path path, final S3AFileStatus fileStatus) { + final Path path, + final S3AFileStatus fileStatus) { return createObjectAttributes( path, fileStatus.getETag(), @@ -1237,7 +1240,7 @@ public FSDataOutputStream create(Path f, FsPermission permission, entryPoint(INVOCATION_CREATE); final Path path = qualify(f); String key = pathToKey(path); - FileStatus status; + FileStatus status = null; try { // get the status or throw an FNFE. // when overwriting, there is no need to look for any existing file, @@ -4845,7 +4848,8 @@ public CompletableFuture openFileWithOptions( openFileHelper.prepareToOpenFile( path, parameters, - getDefaultBlockSize(path)); + getDefaultBlockSize(path), + inputPolicy); boolean isSelect = fileInformation.isSql(); CompletableFuture result = new CompletableFuture<>(); Configuration options = parameters.getOptions(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index d6a281c8ebaa8..074b00650027f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -693,7 +693,7 @@ public String toString() { sb.append(" contentRangeFinish=").append(contentRangeFinish); sb.append(" remainingInCurrentRequest=") .append(remainingInCurrentRequest()); - sb.append(changeTracker); + sb.append(" ").append(changeTracker); sb.append('\n').append(s); sb.append('}'); return sb.toString(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java index d4078dfedf425..2692e64ce4824 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java @@ -48,7 +48,7 @@ * Helper class for openFile() logic, especially processing file status * args and length/etag/versionID. *

- * This got complex enough it merited removal from S3AFileSystemy -which + * This got complex enough it merited removal from S3AFileSystem -which * also permits unit testing. *

*

@@ -63,9 +63,6 @@ public class S3AOpenFileOperation extends AbstractStoreOperation { private static final Logger LOG = LoggerFactory.getLogger(S3AOpenFileOperation.class); - /** Default input policy. */ - private final S3AInputPolicy defaultInputPolicy; - /** Default change detection policy. */ private final ChangeDetectionPolicy changePolicy; @@ -80,20 +77,17 @@ public class S3AOpenFileOperation extends AbstractStoreOperation { /** * Instantiate with the default options from the filesystem. - * @param defaultInputPolicy input policy * @param changePolicy change detection policy * @param defaultReadAhead read ahead range * @param username username * @param defaultBufferSize buffer size */ public S3AOpenFileOperation( - final S3AInputPolicy defaultInputPolicy, final ChangeDetectionPolicy changePolicy, final long defaultReadAhead, final String username, final int defaultBufferSize) { super(null); - this.defaultInputPolicy = defaultInputPolicy; this.changePolicy = changePolicy; this.defaultReadAhead = defaultReadAhead; this.username = username; @@ -199,7 +193,8 @@ public String toString() { public OpenFileInformation prepareToOpenFile( final Path path, final OpenFileParameters parameters, - final long blockSize) throws IOException { + final long blockSize, + final S3AInputPolicy inputPolicy) throws IOException { Configuration options = parameters.getOptions(); Set mandatoryKeys = parameters.getMandatoryKeys(); String sql = options.get(SelectConstants.SELECT_SQL, null); @@ -217,6 +212,9 @@ public OpenFileInformation prepareToOpenFile( InternalConstants.S3A_OPENFILE_KEYS, "for " + path + " in non-select file I/O"); } + + // was a status passed in via a withStatus() invocation in + // the builder API? FileStatus providedStatus = parameters.getStatus(); S3AFileStatus fileStatus; if (providedStatus != null) { @@ -244,7 +242,10 @@ public OpenFileInformation prepareToOpenFile( LOG.debug("File was opened with a supplied FileStatus;" + " skipping getFileStatus call in open() operation: {}", providedStatus); + + // what type is the status (and hence: what information does it contain?) if (providedStatus instanceof S3AFileStatus) { + // is it an S3AFileSystem status? S3AFileStatus st = (S3AFileStatus) providedStatus; versionId = st.getVersionId(); eTag = st.getETag(); @@ -299,7 +300,7 @@ public OpenFileInformation prepareToOpenFile( } // get the first known policy S3AInputPolicy seekPolicy = S3AInputPolicy.getFirstSupportedPolicy(policies, - defaultInputPolicy); + inputPolicy); // readahead range long readAhead = options.getLong(READAHEAD_RANGE, defaultReadAhead); // buffer size @@ -314,9 +315,10 @@ public OpenFileInformation prepareToOpenFile( * @return the parameters needed to open a file through open(path, bufferSize). * @param bufferSize buffer size */ - public OpenFileInformation openSimpleFile(final int bufferSize) { + public OpenFileInformation openSimpleFile(final int bufferSize, + final S3AInputPolicy inputPolicy) { return new OpenFileInformation(false, null, null, - defaultInputPolicy, changePolicy, defaultReadAhead, bufferSize); + inputPolicy, changePolicy, defaultReadAhead, bufferSize); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java index cff4fcde9fdd2..4c11f31315729 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java @@ -69,7 +69,6 @@ public class TestOpenFileHelper extends HadoopTestBase { private static final S3AOpenFileOperation OPERATION = new S3AOpenFileOperation( - INPUT_POLICY, CHANGE_POLICY, READ_AHEAD_RANGE, USERNAME, @@ -78,7 +77,7 @@ public class TestOpenFileHelper extends HadoopTestBase { @Test public void testSimpleFile() throws Throwable { ObjectAssert - asst = assertFI(OPERATION.openSimpleFile(1024)); + asst = assertFI(OPERATION.openSimpleFile(1024, INPUT_POLICY)); asst.extracting(f -> f.getChangePolicy()) .isEqualTo(CHANGE_POLICY); @@ -313,7 +312,9 @@ public S3AOpenFileOperation.OpenFileInformation prepareToOpenFile( final OpenFileParameters parameters) throws IOException { return OPERATION.prepareToOpenFile(TESTPATH, - parameters, IO_FILE_BUFFER_SIZE_DEFAULT); + parameters, + IO_FILE_BUFFER_SIZE_DEFAULT, + INPUT_POLICY); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java index efd96c4e7387e..a849e6837265a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java @@ -43,12 +43,15 @@ import java.io.EOFException; import java.io.IOException; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; /** - * Look at the performance of S3a operations. + * Look at the performance of S3a Input Stream Reads. */ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase { private static final Logger LOG = LoggerFactory.getLogger( @@ -160,18 +163,16 @@ private FSDataInputStream openDataFile(S3AFileSystem fs, long readahead) throws IOException { int bufferSize = getConf().getInt(KEY_READ_BUFFER_SIZE, DEFAULT_READ_BUFFER_SIZE); - S3AInputPolicy policy = fs.getInputPolicy(); - fs.setInputPolicy(inputPolicy); - try { - FSDataInputStream stream = fs.open(path, bufferSize); - if (readahead >= 0) { - stream.setReadahead(readahead); - } - streamStatistics = getInputStreamStatistics(stream); - return stream; - } finally { - fs.setInputPolicy(policy); + FSDataInputStream stream = awaitFuture(fs.openFile(path) + .opt(FS_OPTION_OPENFILE_FADVISE, + inputPolicy.toString()) + .opt(FS_OPTION_OPENFILE_BUFFER_SIZE, bufferSize) + .build()); + if (readahead >= 0) { + stream.setReadahead(readahead); } + streamStatistics = getInputStreamStatistics(stream); + return stream; } /** From 0e88cecd9ed9ce3b74750ac7064fbeb9d3911872 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 24 Nov 2020 13:46:27 +0000 Subject: [PATCH 08/10] HADOOP-16202. fix test failure; S3A committers to use withFileStatus Passing in the FileStatus to openfile will have tangible benefits in task and job commit for the magic committer, while being harmless for staging committers (unless they stage to S3) In task commit: eliminate one HEAD request per file created in that task attempt In job commit: eliminate one HEAD request per task Change-Id: I20d705067d3d3749c05a854b306888dbf72c83a5 --- .../apache/hadoop/util/JsonSerialization.java | 3 ++- .../hadoop/util/TestJsonSerialization.java | 2 +- .../fs/s3a/commit/CommitOperations.java | 8 +++--- .../fs/s3a/commit/files/PendingSet.java | 26 +++++++++++++++---- .../s3a/commit/files/SinglePendingCommit.java | 20 +++++++++++++- 5 files changed, 48 insertions(+), 11 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java index 76b35d6ccde44..db418dd11f49f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java @@ -18,6 +18,7 @@ package org.apache.hadoop.util; +import javax.annotation.Nullable; import java.io.EOFException; import java.io.File; import java.io.FileNotFoundException; @@ -258,7 +259,7 @@ public T load(FileSystem fs, Path path) throws IOException { * @throws EOFException file status references an empty file * @throws IOException IO problems */ - public T load(FileSystem fs, Path path, FileStatus status) + public T load(FileSystem fs, Path path, @Nullable FileStatus status) throws IOException { if (status != null && status.getLen() == 0) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJsonSerialization.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJsonSerialization.java index fb0a81e55f1f3..f50b2b2fb6c3c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJsonSerialization.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJsonSerialization.java @@ -176,7 +176,7 @@ public void testFileSystemEmptyPath() throws Throwable { Path tempPath = new Path(tempFile.toURI()); LocalFileSystem fs = FileSystem.getLocal(new Configuration()); try { - LambdaTestUtils.intercept(EOFException.class, + LambdaTestUtils.intercept(PathIOException.class, () -> serDeser.load(fs, tempPath)); fs.delete(tempPath, false); LambdaTestUtils.intercept(FileNotFoundException.class, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java index 11eeee9e94096..4b704968d1912 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java @@ -241,7 +241,7 @@ public List locateAllSinglePendingCommits( List> failures = new ArrayList<>(1); for (LocatedFileStatus status : statusList) { try { - commits.add(SinglePendingCommit.load(fs, status.getPath())); + commits.add(SinglePendingCommit.load(fs, status.getPath(), status)); } catch (IOException e) { LOG.warn("Failed to load commit file {}", status.getPath(), e); failures.add(Pair.of(status, e)); @@ -324,10 +324,12 @@ public MaybeIOE abortAllSinglePendingCommits(Path pendingDir, LOG.debug("No files to abort under {}", pendingDir); } while (pendingFiles.hasNext()) { - Path pendingFile = pendingFiles.next().getPath(); + LocatedFileStatus status = pendingFiles.next(); + Path pendingFile = status.getPath(); if (pendingFile.getName().endsWith(CommitConstants.PENDING_SUFFIX)) { try { - abortSingleCommit(SinglePendingCommit.load(fs, pendingFile)); + abortSingleCommit(SinglePendingCommit.load(fs, pendingFile, + status)); } catch (FileNotFoundException e) { LOG.debug("listed file already deleted: {}", pendingFile); } catch (IOException | IllegalArgumentException e) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java index 8ad0342105837..d4d1d89b150b9 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.commit.files; +import javax.annotation.Nullable; import java.io.IOException; import java.io.ObjectInputStream; import java.util.ArrayList; @@ -108,10 +109,7 @@ public static JsonSerialization serializer() { */ public static PendingSet load(FileSystem fs, Path path) throws IOException { - LOG.debug("Reading pending commits in file {}", path); - PendingSet instance = serializer().load(fs, path); - instance.validate(); - return instance; + return load(fs, path, null); } /** @@ -124,7 +122,25 @@ public static PendingSet load(FileSystem fs, Path path) */ public static PendingSet load(FileSystem fs, FileStatus status) throws IOException { - return load(fs, status.getPath()); + return load(fs, status.getPath(), status); + } + + /** + * Load an instance from a file, then validate it. + * @param fs filesystem + * @param path path + * @param status status of file to load + * @return the loaded instance + * @throws IOException IO failure + * @throws ValidationFailure if the data is invalid + */ + public static PendingSet load(FileSystem fs, Path path, + @Nullable FileStatus status) + throws IOException { + LOG.debug("Reading pending commits in file {}", path); + PendingSet instance = serializer().load(fs, path, status); + instance.validate(); + return instance; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java index ee7c7b236c6cb..7f8d3b1c60404 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.commit.files; +import javax.annotation.Nullable; import java.io.IOException; import java.io.ObjectInputStream; import java.io.Serializable; @@ -36,6 +37,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; @@ -137,7 +139,23 @@ public static JsonSerialization serializer() { */ public static SinglePendingCommit load(FileSystem fs, Path path) throws IOException { - SinglePendingCommit instance = serializer().load(fs, path); + return load(fs, path, null); + } + + /** + * Load an instance from a file, then validate it. + * @param fs filesystem + * @param path path + * @param status status of file to load or null + * @return the loaded instance + * @throws IOException IO failure + * @throws ValidationFailure if the data is invalid + */ + public static SinglePendingCommit load(FileSystem fs, + Path path, + @Nullable FileStatus status) + throws IOException { + SinglePendingCommit instance = serializer().load(fs, path, status); instance.filename = path.toString(); instance.validate(); return instance; From 0966a2895aba30f7c084ff7de7e423ef7d4f74f1 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 24 Nov 2020 14:19:21 +0000 Subject: [PATCH 09/10] HADOOP-16202 style and javadoc warnings Change-Id: I1e454d3244ffab8c248584edfb1f539ebac596c7 --- .../hadoop/fs/FutureDataInputStreamBuilder.java | 4 ++-- .../apache/hadoop/fs/impl/AbstractFSBuilderImpl.java | 8 +++----- .../fs/impl/FutureDataInputStreamBuilderImpl.java | 5 +++-- .../hadoop/fs/contract/AbstractContractOpenTest.java | 3 ++- .../mapreduce/jobhistory/JobHistoryCopyService.java | 1 - .../hadoop/fs/s3a/impl/S3AOpenFileOperation.java | 3 ++- .../hadoop/fs/contract/s3a/ITestS3AContractSeek.java | 2 -- .../hadoop/fs/s3a/performance/ITestS3AOpenCost.java | 12 ++++++------ 8 files changed, 18 insertions(+), 20 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java index e2601a0b5ae2c..e7f441a75d3c8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FutureDataInputStreamBuilder.java @@ -28,12 +28,12 @@ * Builder for input streams and subclasses whose return value is * actually a completable future: this allows for better asynchronous * operation. - *

+ * * To be more generic, {@link #opt(String, int)} and {@link #must(String, int)} * variants provide implementation-agnostic way to customize the builder. * Each FS-specific builder implementation can interpret the FS-specific * options accordingly, for example: - *

+ * * If the option is not related to the file system, the option will be ignored. * If the option is must, but not supported/known by the file system, an * {@link IllegalArgumentException} will be thrown. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java index bfecd99928a67..2308e38a1c9e6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java @@ -42,20 +42,18 @@ /** * Builder for filesystem/filecontext operations of various kinds, * with option support. - *

- *
+ *
+ * 
  *   .opt("foofs:option.a", true)
  *   .opt("foofs:option.b", "value")
  *   .opt("fs.s3a.open.option.etag", "9fe4c37c25b")
  *   .must("foofs:cache", true)
  *   .must("barfs:cache-size", 256 * 1024 * 1024)
  *   .build();
- * 
- *

+ *
* * Configuration keys declared in an {@code opt()} may be ignored by * a builder which does not recognise them. - *

* * Configuration keys declared in a {@code must()} function set must * be understood by the implementation or a diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java index 9790b51bd153c..70e39de7388c3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java @@ -41,15 +41,16 @@ * Builder for input streams and subclasses whose return value is * actually a completable future: this allows for better asynchronous * operation. - *

+ * * To be more generic, {@link #opt(String, int)} and {@link #must(String, int)} * variants provide implementation-agnostic way to customize the builder. * Each FS-specific builder implementation can interpret the FS-specific * options accordingly, for example: - *

+ * * If the option is not related to the file system, the option will be ignored. * If the option is must, but not supported/known by the file system, an * {@link IllegalArgumentException} will be thrown. + * */ @InterfaceAudience.Public @InterfaceStability.Unstable diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java index c3eb8adbc7c23..a4419be7937ad 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java @@ -343,7 +343,8 @@ public void testOpenFileApplyAsyncRead() throws Throwable { @Test public void testOpenFileNullStatusButFileLength() throws Throwable { describe("use openFile() with a null status and expect the status to be" - + " ignored. block size, fadvise and length are passed in as must() options"); + + " ignored. block size, fadvise and length are passed in as" + + " must() options"); Path path = path("testOpenFileNullStatus"); FileSystem fs = getFileSystem(); int len = 4; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java index 792e1c2a124d3..33e4ecafabdc5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java @@ -37,7 +37,6 @@ import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; -import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java index 2692e64ce4824..cdf99b5205d28 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java @@ -312,7 +312,8 @@ public OpenFileInformation prepareToOpenFile( /** * Open a simple file. - * @return the parameters needed to open a file through open(path, bufferSize). + * @return the parameters needed to open a file through + * {@code open(path, bufferSize)}. * @param bufferSize buffer size */ public OpenFileInformation openSimpleFile(final int bufferSize, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java index e906fc2b23d24..3088995e999db 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java @@ -24,7 +24,6 @@ import java.util.Arrays; import java.util.Collection; -import org.assertj.core.api.Assertions; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -45,7 +44,6 @@ import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.NativeCodeLoader; -import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADV_NORMAL; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java index 85d7fc0003149..06c98bb3372ca 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java @@ -194,12 +194,12 @@ public void testOpenFileLongerLength() throws Throwable { byte[] out = new byte[(int) longLen]; intercept(EOFException.class, () -> in3.readFully(0, out)); - in3.seek(longLen - 1); - assertEquals("read past real EOF on " + in3, - -1, in3.read()); - in3.close(); - return in3.toString(); - }, + in3.seek(longLen - 1); + assertEquals("read past real EOF on " + in3, + -1, in3.read()); + in3.close(); + return in3.toString(); + }, // two GET calls were made, one for readFully, // the second on the read() past the EOF // the operation has got as far as S3 From 194c1fdc64aaeffea7c01a4d1e3ebfc0a532cbea Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 3 Dec 2020 17:37:36 +0000 Subject: [PATCH 10/10] HADOOP-16202. add explicit split start/end options. I had the great idea of adding explict options to declare the split.start and split.end, where split end can be used to define the file length. So I added them and wired up the relevant bits of the MR code. But: we can't actually rely on them because some readers may read past the end of the split to get to the end of their record. It has to be viewed as a hint "we will read this far" not a rule "we will read this far and no further". This is reflected in the docs and tests for s3a. This actually complicates the story for opening files efficiently in MR jobs/hive as the split size is passed down but not the final file length. If we want to eliminate those head requests we really need the length. Change-Id: I44e05d45881155ebcd9407470b17f1c0d89c8952 --- .../java/org/apache/hadoop/fs/Options.java | 20 +- .../src/site/markdown/filesystem/openfile.md | 46 ++- .../hadoop/mapred/LineRecordReader.java | 7 + .../mapreduce/lib/input/LineRecordReader.java | 7 + .../examples/terasort/TeraInputFormat.java | 16 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 13 +- .../hadoop/fs/s3a/S3ObjectAttributes.java | 18 +- .../fs/s3a/impl/S3AOpenFileOperation.java | 298 +++++++++++------- .../fs/s3a/TestStreamChangeTracker.java | 2 +- .../fs/s3a/impl/TestOpenFileHelper.java | 48 ++- 10 files changed, 346 insertions(+), 129 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java index a9e7bcd90676b..e93f532c9bca5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java @@ -545,11 +545,23 @@ private OpenFileOptions() { FILESYSTEM_OPTION + "openfile."; /** - * OpenFile option for seek policies: {@value}. + * 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}. */ @@ -587,14 +599,16 @@ private OpenFileOptions() { "adaptive"; /** - * Set of standard options which openfile implementations + * Set of standard options which openFile implementations * MUST recognize, even if they ignore the actual values. */ public static final Set FS_OPTION_OPENFILE_STANDARD_OPTIONS = Collections.unmodifiableSet(Stream.of( FS_OPTION_OPENFILE_BUFFER_SIZE, FS_OPTION_OPENFILE_FADVISE, - FS_OPTION_OPENFILE_LENGTH) + FS_OPTION_OPENFILE_LENGTH, + FS_OPTION_OPENFILE_SPLIT_START, + FS_OPTION_OPENFILE_SPLIT_END) .collect(Collectors.toSet())); } diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md index 353b28d5a18d6..adbed68209887 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md @@ -180,7 +180,6 @@ performed efficiently or not. Declare the length of a file. This MAY be used to skip a probe for the file length. - If supported by a filesystem connector, this option MUST be interpreted as declaring the minimum length of the file: @@ -189,6 +188,22 @@ the minimum length of the file: but below the actual length of the file. Implementations MAY raise `EOFExceptions` in such cases, or they MAY return data. +#### `fs.option.openfile.split.start` and `fs.option.openfile.split.end` + +Declare the start and end of the split when a file has been split for processing in pieces. + +1. Filesystems MAY assume that the length of the file is greater than or equal to +the value of `fs.option.openfile.split.end`. +1. And that they MAY raise an exception if the client application + reads past the value set in `fs.option.openfile.split.end`. +1. The pair of options MAY be used to optimise the read plan, such as setting the content +range for GET requests, or using the split end as an implicit declaration of the guaranteed +minimum length of the file. +1. If both options are set, and the split start is declared as greater than the split end, then the split start SHOULD just be reset to zero, rather than rejecting the operation. + +The split end value is the most useful as it can delinate the end of the input stream. +The split start can be used to optimize any initial read offset for filesystem clients. + ## Example @@ -206,10 +221,8 @@ protected SeekableInputStream newStream(Path path, FileStatus stat, .opt("fs.option.openfile.fadvise", "random") .withFileStatus(stat); - // fall back to leng - if (stat == null && splitEnd > 0) { - builder.opt("fs.option.openfile.length", splitEnd); - } + builder.opt("fs.option.openfile.split.start", splitStart); + builder.opt("fs.option.openfile.split.end", splitEnd); CompletableFuture streamF = builder.build(); return HadoopStreams.wrap(awaitFuture(streamF)); } @@ -221,10 +234,31 @@ there is no need to probe the remote store for the length of the file. When working with remote object stores, this can save tens to hundreds of milliseconds, even if such a probe is done asynchronously. +If both the file length and the split end is set, then the file length should be considered "more" authoritative, that is it really SHOULD be defining the file length. +However, if the split is set, the caller SHOULD not read past it. + +There's two issues which surface here: + +* What if both length and split end are set: which defines the length of the read? +* Do applications ever read past the end of the split? + +The `CompressedSplitLineReader` can read past the end of a split if it is partway through processing a compressed record. +That is: it assumes an incomplete record read means that the file length is greater than the split length, and that it MUST read the entirety of the partially read record. +Other readers may behave similarly. + +Therefore + +1. File length as supplied in a `FileStatus` or in `fs.option.openfile.length` SHALL set the strict upper limit on the length of a file +2. The split end as set in `fs.option.openfile.split.end` MUST be viewed as a hint, rather than the strict end of the file. + +This reduces the value of the split start/end settings. +They cannot be treated as normative declarations of the maximum read of a stream. + + ## S3A Non-standard options -The S3A Connector supports custom options +The S3A Connector supports custom options | Name | Type | Meaning | diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java index 1fcb118a100fc..21e1ffe716705 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java @@ -44,6 +44,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START; + /** * Treats keys as offset in file and value as line. */ @@ -109,6 +112,10 @@ public LineRecordReader(Configuration job, FileSplit split, // open the file and seek to the start of the split final FutureDataInputStreamBuilder builder = 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) + .opt(FS_OPTION_OPENFILE_SPLIT_END, end); FutureIOSupport.propagateOptions(builder, job, MRJobConfig.INPUT_FILE_OPTION_PREFIX, MRJobConfig.INPUT_FILE_MANDATORY_PREFIX); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java index 160c7635658a4..93d8ceb4d04ad 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java @@ -43,6 +43,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START; + /** * Treats keys as offset in file and value as line. */ @@ -86,6 +89,10 @@ public void initialize(InputSplit genericSplit, // open the file and seek to the start of the split final FutureDataInputStreamBuilder builder = 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); FutureIOSupport.propagateOptions(builder, job, MRJobConfig.INPUT_FILE_OPTION_PREFIX, MRJobConfig.INPUT_FILE_MANDATORY_PREFIX); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java index 20ce8ef2b60de..a290fb07d9efa 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java @@ -27,7 +27,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.JobContext; @@ -42,6 +44,11 @@ import org.apache.hadoop.util.QuickSort; import org.apache.hadoop.util.StringUtils; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START; + /** * An input format that reads the first 10 characters of each line as the key * and the rest of the line as the value. Both key and value are represented @@ -224,12 +231,17 @@ public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { Path p = ((FileSplit)split).getPath(); FileSystem fs = p.getFileSystem(context.getConfiguration()); - in = fs.open(p); long start = ((FileSplit)split).getStart(); // find the offset to start at a record boundary offset = (RECORD_LENGTH - (start % RECORD_LENGTH)) % RECORD_LENGTH; - in.seek(start + offset); length = ((FileSplit)split).getLength(); + final FutureDataInputStreamBuilder builder = fs.openFile(p) + .opt(FS_OPTION_OPENFILE_SPLIT_START, start) + .opt(FS_OPTION_OPENFILE_SPLIT_END, start+ length) + .opt(FS_OPTION_OPENFILE_FADVISE, + FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL); + in = FutureIOSupport.awaitFuture(builder.build()); + in.seek(start + offset); } public void close() throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 001172a833462..f8c7f94de9259 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -1124,9 +1124,9 @@ public FSDataInputStream open(Path f, int bufferSize) /** * Opens an FSDataInputStream at the indicated Path. - * if fileInformation contains an FileStatus reference, - * then that is used to avoid a check for the file length/existence - * + * The {@code fileInformation} parameter controls how the file + * is opened, whether it is normal vs. an S3 select call, + * can a HEAD be skipped, etc. * @param path the file to open * @param fileInformation information about the file to open * @throws IOException IO failure. @@ -1197,6 +1197,8 @@ private S3ObjectAttributes createObjectAttributes( encryptionSecrets.getEncryptionKey(), eTag, versionId, + len, + 0, len); } @@ -1560,7 +1562,8 @@ public S3ObjectAttributes createObjectAttributes( public S3AReadOpContext createReadContext(final FileStatus fileStatus) { return S3AFileSystem.this.createReadContext(fileStatus, inputPolicy, - changeDetectionPolicy, readAhead); + changeDetectionPolicy, + readAhead); } @Override @@ -4828,7 +4831,7 @@ private S3AFileStatus extractOrFetchSimpleFileStatus( } /** - * Initiate the open or select operation. + * Initiate the open() or select() operation. * This is invoked from both the FileSystem and FileContext APIs * @param rawPath path to the file * @param parameters open file parameters from the builder. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java index 5a8dfc7bc8c0e..00b367c1d62c2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java @@ -43,6 +43,8 @@ public class S3ObjectAttributes { private final String eTag; private final String versionId; private final long len; + private final long readStart; + private final long readEnd; public S3ObjectAttributes( String bucket, @@ -52,7 +54,9 @@ public S3ObjectAttributes( String serverSideEncryptionKey, String eTag, String versionId, - long len) { + long len, + long readStart, + long readEnd) { this.bucket = bucket; this.path = path; this.key = key; @@ -61,6 +65,8 @@ public S3ObjectAttributes( this.eTag = eTag; this.versionId = versionId; this.len = len; + this.readStart = readStart; + this.readEnd = readEnd; } /** @@ -86,6 +92,8 @@ public S3ObjectAttributes( this.eTag = copyResult.getETag(); this.versionId = copyResult.getVersionId(); this.len = len; + this.readStart = 0; + this.readEnd = len; } public String getBucket() { @@ -119,4 +127,12 @@ public long getLen() { public Path getPath() { return path; } + + public long getReadStart() { + return readStart; + } + + public long getReadEnd() { + return readEnd; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java index cdf99b5205d28..6e06cfa2f7b60 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AOpenFileOperation.java @@ -39,6 +39,8 @@ import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START; import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; @@ -63,6 +65,12 @@ public class S3AOpenFileOperation extends AbstractStoreOperation { private static final Logger LOG = LoggerFactory.getLogger(S3AOpenFileOperation.class); + + /** + * For use when a value of an split/file length is unknown. + */ + private static final int LENGTH_UNKNOWN = -1; + /** Default change detection policy. */ private final ChangeDetectionPolicy changePolicy; @@ -94,102 +102,17 @@ public S3AOpenFileOperation( this.defaultBufferSize = defaultBufferSize; } - /** - * The information on a file needed to open it. - */ - public static final class OpenFileInformation { - - /** Is this SQL? */ - private final boolean isSql; - - /** File status; may be null. */ - private final S3AFileStatus status; - - /** SQL string if this is a SQL select file. */ - private final String sql; - - /** Active input policy. */ - private final S3AInputPolicy inputPolicy; - - /** Change detection policy. */ - private final ChangeDetectionPolicy changePolicy; - - /** read ahead range. */ - private final long readAheadRange; - - /** Buffer size. Currently ignored. */ - private final int bufferSize; - - /** - * Constructor. - */ - private OpenFileInformation(final boolean isSql, - final S3AFileStatus status, - final String sql, - final S3AInputPolicy inputPolicy, - final ChangeDetectionPolicy changePolicy, - final long readAheadRange, - final int bufferSize) { - this.isSql = isSql; - this.status = status; - this.sql = sql; - this.inputPolicy = inputPolicy; - this.changePolicy = changePolicy; - this.readAheadRange = readAheadRange; - this.bufferSize = bufferSize; - } - - public boolean isSql() { - return isSql; - } - - public S3AFileStatus getStatus() { - return status; - } - - public String getSql() { - return sql; - } - - public S3AInputPolicy getInputPolicy() { - return inputPolicy; - } - - public ChangeDetectionPolicy getChangePolicy() { - return changePolicy; - } - - public long getReadAheadRange() { - return readAheadRange; - } - - public int getBufferSize() { - return bufferSize; - } - - @Override - public String toString() { - return "OpenFileInformation{" + - "isSql=" + isSql + - ", status=" + status + - ", sql='" + sql + '\'' + - ", inputPolicy=" + inputPolicy + - ", changePolicy=" + changePolicy + - ", readAheadRange=" + readAheadRange + - ", bufferSize=" + bufferSize + - '}'; - } - } - /** * Prepare to open a file from the openFile parameters. * @param path path to the file * @param parameters open file parameters from the builder. * @param blockSize for fileStatus + * @param inputPolicy default input policy. * @return open file options * @throws IOException failure to resolve the link. * @throws IllegalArgumentException unknown mandatory key */ + @SuppressWarnings("ChainOfInstanceofChecks") public OpenFileInformation prepareToOpenFile( final Path path, final OpenFileParameters parameters, @@ -213,10 +136,13 @@ public OpenFileInformation prepareToOpenFile( "for " + path + " in non-select file I/O"); } + // where does a read end? + long fileLength = LENGTH_UNKNOWN; + // was a status passed in via a withStatus() invocation in // the builder API? FileStatus providedStatus = parameters.getStatus(); - S3AFileStatus fileStatus; + S3AFileStatus fileStatus = null; if (providedStatus != null) { // there's a file status @@ -271,24 +197,30 @@ public OpenFileInformation prepareToOpenFile( username, eTag, versionId); - } else if (options.get(FS_OPTION_OPENFILE_LENGTH) != null) { - // build a minimal S3A FileStatus From the input length alone. - // this is all we actually need. - long length = options.getLong(FS_OPTION_OPENFILE_LENGTH, 0); - LOG.debug("Fixing length of file to read {} as {}", path, length); - fileStatus = new S3AFileStatus( - length, - 0, - path, - blockSize, - username, - null, - null); - } else { - // neither a file status nor a file length - fileStatus = null; + // set the end of the read to the file length + fileLength = fileStatus.getLen(); + } + // determine start and end of file. + long splitStart = options.getLong(FS_OPTION_OPENFILE_SPLIT_START, 0); + + // split end + long splitEnd = options.getLong(FS_OPTION_OPENFILE_SPLIT_END, + LENGTH_UNKNOWN); + if (splitStart > splitEnd) { + LOG.warn("Split start {} is greater than split end {}, resetting", + splitStart, splitEnd); + splitStart = 0; } + // read end is the open file value + fileLength = options.getLong(FS_OPTION_OPENFILE_LENGTH, fileLength); + + // if the read end has come from options, use that + // in creating a file status + if (fileLength >= 0 && fileStatus == null) { + fileStatus = createStatus(path, fileLength, blockSize); + } + // Build up the input policy. // seek policy from default, s3a opt or standard option // read from the FS standard option. @@ -307,19 +239,165 @@ public OpenFileInformation prepareToOpenFile( int bufferSize = options.getInt(FS_OPTION_OPENFILE_BUFFER_SIZE, defaultBufferSize); return new OpenFileInformation(isSelect, fileStatus, sql, seekPolicy, - changePolicy, readAhead, bufferSize); + changePolicy, readAhead, bufferSize, splitStart, splitEnd, fileLength); } /** - * Open a simple file. - * @return the parameters needed to open a file through - * {@code open(path, bufferSize)}. - * @param bufferSize buffer size + * Create a minimal file status. + * @param path path + * @param length file length/read end + * @param blockSize block size + * @return a new status */ + private S3AFileStatus createStatus(Path path, long length, long blockSize) { + return new S3AFileStatus( + length, + 0, + path, + blockSize, + username, + null, + null); + } + + + /** + * Open a simple file. + * @return the parameters needed to open a file through + * {@code open(path, bufferSize)}. + * @param bufferSize buffer size + * @param inputPolicy input policy. + */ public OpenFileInformation openSimpleFile(final int bufferSize, final S3AInputPolicy inputPolicy) { return new OpenFileInformation(false, null, null, - inputPolicy, changePolicy, defaultReadAhead, bufferSize); + inputPolicy, changePolicy, defaultReadAhead, bufferSize, + 0, LENGTH_UNKNOWN, LENGTH_UNKNOWN); + } + + /** + * The information on a file needed to open it. + */ + public static final class OpenFileInformation { + + /** Is this SQL? */ + private final boolean isSql; + + /** File status; may be null. */ + private final S3AFileStatus status; + + /** SQL string if this is a SQL select file. */ + private final String sql; + + /** Active input policy. */ + private final S3AInputPolicy inputPolicy; + + /** Change detection policy. */ + private final ChangeDetectionPolicy changePolicy; + + /** Read ahead range. */ + private final long readAheadRange; + + /** Buffer size. Currently ignored. */ + private final int bufferSize; + + /** + * Where does the read start from. 0 unless known. + */ + private final long splitStart; + + /** + * What is the split end? + * Negative if not known. + */ + private final long splitEnd; + + /** + * What is the file length? + * Negative if not known. + */ + private final long fileLength; + + /** + * Constructor. + */ + private OpenFileInformation( + final boolean isSql, + final S3AFileStatus status, + final String sql, + final S3AInputPolicy inputPolicy, + final ChangeDetectionPolicy changePolicy, + final long readAheadRange, + final int bufferSize, + final long splitStart, + final long splitEnd, + final long fileLength) { + this.isSql = isSql; + this.status = status; + this.sql = sql; + this.inputPolicy = inputPolicy; + this.changePolicy = changePolicy; + this.readAheadRange = readAheadRange; + this.bufferSize = bufferSize; + this.splitStart = splitStart; + this.splitEnd = splitEnd; + this.fileLength = fileLength; + } + + public boolean isSql() { + return isSql; + } + + public S3AFileStatus getStatus() { + return status; + } + + public String getSql() { + return sql; + } + + public S3AInputPolicy getInputPolicy() { + return inputPolicy; + } + + public ChangeDetectionPolicy getChangePolicy() { + return changePolicy; + } + + public long getReadAheadRange() { + return readAheadRange; + } + + public int getBufferSize() { + return bufferSize; + } + + public long getSplitStart() { + return splitStart; + } + + public long getSplitEnd() { + return splitEnd; + } + + @Override + public String toString() { + return "OpenFileInformation{" + + "isSql=" + isSql + + ", status=" + status + + ", sql='" + sql + '\'' + + ", inputPolicy=" + inputPolicy + + ", changePolicy=" + changePolicy + + ", readAheadRange=" + readAheadRange + + ", splitStart=" + splitStart + + ", splitEnd=" + splitEnd + + ", bufferSize=" + bufferSize + + '}'; + } + + public long getFileLength() { + return fileLength; + } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index 3d7cdfc08dec4..aeba2ac8656b5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -443,6 +443,6 @@ private S3ObjectAttributes objectAttributes( null, etag, versionId, - 0); + 0, 0, 0); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java index 4c11f31315729..3f4b1ba8000bc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileHelper.java @@ -21,6 +21,9 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; import org.assertj.core.api.Assertions; import org.assertj.core.api.ObjectAssert; @@ -43,6 +46,8 @@ import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_RANDOM; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FADVISE_SEQUENTIAL; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -205,7 +210,7 @@ public void testUnknownSeekPolicyFallback() throws Throwable { * Test the mapping of the standard option names. */ @Test - public void testWellKnownPolicyMapping() throws Throwable { + public void testInputPolicyMapping() throws Throwable { Object[][] policyMapping = { {"normal", S3AInputPolicy.Normal}, {FS_OPTION_OPENFILE_FADVISE_NORMAL, S3AInputPolicy.Normal}, @@ -336,6 +341,47 @@ public void testFileLength() throws Throwable { } + /** + * Verify that setting the split end sets the length. + * By passing in a value greater than the size of an int, + * the test verifies that the long is passed everywhere. + */ + @Test + public void testSplitEndSetsLength() throws Throwable { + long bigFile = 2L ^ 34; + assertOpenFile(FS_OPTION_OPENFILE_SPLIT_END, Long.toString(bigFile)) + .matches(p -> p.getSplitEnd() == bigFile, "split end") + .matches(p -> p.getFileLength() == -1, "file length") + .matches(p -> p.getStatus() == null, "status"); + } + + /** + * Semantics of split and length. Split end can only be safely treated + * as a hint unless the codec is known (how?) that it will never + * read past it. + */ + @Test + public void testSplitEndAndLength() throws Throwable { + long splitEnd = 256; + long len = 8192; + Configuration conf = conf(FS_OPTION_OPENFILE_LENGTH, + Long.toString(len)); + conf.setLong(FS_OPTION_OPENFILE_SPLIT_END, splitEnd); + conf.setLong(FS_OPTION_OPENFILE_SPLIT_START, 1024); + Set s = new HashSet<>(); + Collections.addAll(s, + FS_OPTION_OPENFILE_SPLIT_START, + FS_OPTION_OPENFILE_SPLIT_END, + FS_OPTION_OPENFILE_LENGTH); + assertFI(prepareToOpenFile( + new OpenFileParameters() + .withMandatoryKeys(s) + .withOptions(conf))) + .matches(p -> p.getSplitStart() == 0, "split start") + .matches(p -> p.getSplitEnd() == splitEnd, "split end") + .matches(p -> p.getStatus().getLen() == len, "file length"); + } + /** * Create an S3A status entry with stub etag and versions, timestamp of 0. * @param path status path