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

GH-3078: Use Hadoop FileSystem.openFile() to open files #3079

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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 @@ -19,8 +19,13 @@

package org.apache.parquet.hadoop.util;

import static org.apache.parquet.hadoop.util.wrapped.io.FutureIO.awaitFuture;

import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.concurrent.CompletableFuture;
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.Path;
Expand All @@ -29,6 +34,24 @@

public class HadoopInputFile implements InputFile {

/**
* openFile() option name for setting the read policy: {@value}.
*/
private static final String OPENFILE_READ_POLICY_KEY = "fs.option.openfile.read.policy";

/**
* Read policy when opening parquet files: {@value}.
* <p>Policy-aware stores pick the first policy they recognize in the list.
* everything recognizes "random";
* "vector" came in with 3.4.0, while "parquet" came with Hadoop 3.4.1
* parquet means "this is a Parquet file, so be clever about footers, prefetch,
* and expect vector and/or random IO".
* <p>In Hadoop 3.4.1, "parquet" and "vector" are both mapped to "random" for the
* S3A connector, but as the ABFS and GCS connectors do footer caching, they
* may use it as a hint to say "fetch the footer and keep it in memory"
*/
private static final String PARQUET_READ_POLICY = "parquet, vector, random, adaptive";

private final FileSystem fs;
private final FileStatus stat;
private final Configuration conf;
Expand Down Expand Up @@ -70,9 +93,38 @@ public long getLength() {
return stat.getLen();
}

/**
* Open the file.
* <p>Uses {@code FileSystem.openFile()} so that
* the existing FileStatus can be passed down: saves a HEAD request on cloud storage.
* and ignored everywhere else.
*
* @return the input stream.
*
* @throws InterruptedIOException future was interrupted
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
*/
@Override
public SeekableInputStream newStream() throws IOException {
return HadoopStreams.wrap(fs.open(stat.getPath()));
FSDataInputStream stream;
try {
// this method is async so that implementations may do async HEAD head
// requests. Not done in S3A/ABFS when a file status passed down (as is done here)
final CompletableFuture<FSDataInputStream> future = fs.openFile(stat.getPath())
.withFileStatus(stat)
.opt(OPENFILE_READ_POLICY_KEY, PARQUET_READ_POLICY)
.build();
stream = awaitFuture(future);
} catch (RuntimeException e) {
// S3A < 3.3.5 would raise illegal path exception if the openFile path didn't
// equal the path in the FileStatus; Hive virtual FS could create this condition.
// As the path to open is derived from stat.getPath(), this condition seems
// near-impossible to create -but is handled here for due diligence.
stream = fs.open(stat.getPath());
Copy link
Contributor

Choose a reason for hiding this comment

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

Shouldn't we at least log the original exception?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

good point. at debug? I'm never a fan of excessively noisy libraries as they ruin the lives of people downstream, especially if messages for developers end up being printed in user logs far too often. In fact, I'm not above dynamically patching log levels to shut those libraries up (HADOOP-19272)...

stack trace then, debug.
error text itself? Probably the same unless there's a desire to log exactly once during the life of a process, which could be done trivially

Copy link
Contributor

Choose a reason for hiding this comment

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

I'm fine with any solutions just did not want to loose the exception may occur.

Another idea I've had is to catch the potential exception may come out of this line (let's call it e2) and add the previously caught e as suppressed to e2, then re-throw. But it might be an overkill since this code path "near-impossible" to be taken.

}

return HadoopStreams.wrap(stream);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,29 @@ public static <T> T awaitFuture(final Future<T> future, final long timeout, fina
}
}

/**
* Given a future, evaluate it.
* <p>
* Any exception generated in the future is
* extracted and rethrown.
* </p>
* @param future future to evaluate
* @param <T> type of the result.
* @return the result, if all went well.
* @throws InterruptedIOException future was interrupted
* @throws IOException if something went wrong
* @throws RuntimeException any nested RTE thrown
*/
public static <T> T awaitFuture(final Future<T> future)
throws InterruptedIOException, IOException, RuntimeException {
try {
return future.get();
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException(e.toString()).initCause(e);
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: Why the cast?

Copy link
Contributor Author

@steveloughran steveloughran Nov 29, 2024

Choose a reason for hiding this comment

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

initCause() returns a throwable; you need to casr it back to what the actual exception is so you can throw it

Copy link
Contributor

Choose a reason for hiding this comment

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

Oh, missed that, thanks!

} catch (ExecutionException e) {
throw unwrapInnerException(e);
}
}
/**
* From the inner cause of an execution exception, extract the inner cause
* to an IOException, raising Errors immediately.
Expand Down