Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HADOOP-16202. Enhance openFile() for better read performance against object stores #2584

Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,10 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;

/** Adapts an {@link FSDataInputStream} to Avro's SeekableInput interface. */
@InterfaceAudience.Public
@InterfaceStability.Stable
Expand All @@ -42,7 +46,12 @@ 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_READ_POLICY,
FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL)
.withFileStatus(status)
.build());
fc.open(p);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -45,6 +44,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;
import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable;

Expand Down Expand Up @@ -889,7 +889,7 @@ protected CompletableFuture<FSDataInputStream> openFileWithOptions(
final OpenFileParameters parameters) throws IOException {
AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
parameters.getMandatoryKeys(),
Collections.emptySet(),
FS_OPTION_OPENFILE_STANDARD_OPTIONS,
Comment on lines 891 to +892
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With this change, ChecksumFileSystem.openFileWithOptions has the same implementation as the base class, so you can remove this override.

"for " + path);
return LambdaUtils.eval(
new CompletableFuture<>(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,13 @@ public interface FSBuilder<S, B extends FSBuilder<S, B>> {
*/
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);

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We could use properties (getter and setter) for the standard options. We'd have one for buffer size, one for file length, etc.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree, but as well as it already being out there, I want to let applications compile against any version of hadoop with the API, even if a specific FS option isn't available, alongside allowing for custom FS opts. As an example, I have a PoC of parquet lib which uses this which is designed to compile against 3.3.x. (that isn't something I've stuck up as I use it to see how this stuff could be added to a library...highlights what is broken right now, specifically; S3A openFile.withFileStatus fails if used via Hive because hive wraps the FileStatus to a different type from S3AFileStatus.

/**
* Set optional double parameter for the Builder.
*
Expand Down Expand Up @@ -104,6 +111,13 @@ public interface FSBuilder<S, B extends FSBuilder<S, B>> {
*/
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.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,12 @@
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_READ_POLICY;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;

/**
* The FileContext class provides an interface for users of the Hadoop
Expand Down Expand Up @@ -2198,7 +2203,12 @@ public boolean copy(final Path src, final Path dst, boolean deleteSource,
EnumSet<CreateFlag> 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_READ_POLICY,
FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
.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 {
Expand Down Expand Up @@ -2930,9 +2940,11 @@ public CompletableFuture<FSDataInputStream> build() throws IOException {
final Path absF = fixRelativePart(getPath());
OpenFileParameters parameters = new OpenFileParameters()
.withMandatoryKeys(getMandatoryKeys())
.withOptionalKeys(getOptionalKeys())
.withOptions(getOptions())
.withBufferSize(getBufferSize())
.withStatus(getStatus());
.withStatus(getStatus())
.withBufferSize(
getOptions().getInt(FS_OPTION_OPENFILE_BUFFER_SIZE, getBufferSize()));
return new FSLinkResolver<CompletableFuture<FSDataInputStream>>() {
@Override
public CompletableFuture<FSDataInputStream> next(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,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.util.Preconditions.checkArgument;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
Expand Down Expand Up @@ -4616,7 +4617,7 @@ protected CompletableFuture<FSDataInputStream> 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<>(), () ->
Expand Down Expand Up @@ -4644,7 +4645,7 @@ protected CompletableFuture<FSDataInputStream> openFileWithOptions(
final OpenFileParameters parameters) throws IOException {
AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This rejectUnknownMandatoryKeys function would not be necessary if the mandatory keys were strongly typed fields.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree, but we need flexibility of linking, ability of clients to work with any FS implementation, etc etc

parameters.getMandatoryKeys(),
Collections.emptySet(), "");
Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS, "");
CompletableFuture<FSDataInputStream> result = new CompletableFuture<>();
try {
result.complete(open(pathHandle, parameters.getBufferSize()));
Expand Down Expand Up @@ -4751,9 +4752,11 @@ public CompletableFuture<FSDataInputStream> build() throws IOException {
Optional<Path> optionalPath = getOptionalPath();
OpenFileParameters parameters = new OpenFileParameters()
.withMandatoryKeys(getMandatoryKeys())
.withOptionalKeys(getOptionalKeys())
.withOptions(getOptions())
.withBufferSize(getBufferSize())
.withStatus(super.getStatus()); // explicit to avoid IDE warnings
.withStatus(super.getStatus())
.withBufferSize(
getOptions().getInt(FS_OPTION_OPENFILE_BUFFER_SIZE, getBufferSize()));
if(optionalPath.isPresent()) {
return getFS().openFileWithOptions(optionalPath.get(),
parameters);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;

/**
* A collection of file-processing util methods
*/
Expand Down Expand Up @@ -396,7 +401,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.
* <p></p>
* 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.
* <p></p>
* 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").
* <p></p>
* 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.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is the benefit of returning true on success and throwing on a failure, as opposed to returning void and throwing on failure?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

none. its just always been that way. and yes, it is wrong

* @throws IOException failure
*/
public static boolean copy(FileSystem srcFS, FileStatus srcStatus,
FileSystem dstFS, Path dst,
boolean deleteSource,
Expand All @@ -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<FileStatus> 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_READ_POLICY,
FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
.opt(FS_OPTION_OPENFILE_LENGTH,
srcStatus.getLen()) // file length hint for object stores
Comment on lines +456 to +457
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When should we use FS_OPTION_OPENFILE_LENGTH option vs. .withFileStatus(status)?

.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;
}
}
Expand Down Expand Up @@ -503,7 +538,11 @@ private static boolean copy(FileSystem srcFS, FileStatus srcStatus,
deleteSource, conf);
}
} else {
InputStream in = srcFS.open(src);
InputStream in = awaitFuture(srcFS.openFile(src)
.withFileStatus(srcStatus)
.opt(FS_OPTION_OPENFILE_READ_POLICY,
FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
.build());
IOUtils.copyBytes(in, Files.newOutputStream(dst.toPath()), conf);
}
if (deleteSource) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
*/
package org.apache.hadoop.fs;

import javax.annotation.Nullable;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;

Expand All @@ -34,7 +35,7 @@
* 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.
*
*/
Expand All @@ -51,10 +52,11 @@ CompletableFuture<FSDataInputStream> 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;
}

Expand Down
Loading