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

Merged
merged 2 commits into from
Dec 4, 2024

Conversation

steveloughran
Copy link
Contributor

@steveloughran steveloughran commented Nov 26, 2024

Rationale for this change

If parquet passes down the FileStatus it previously collected, S3A and ABFS clients can skip their own HEAD probe. This saves time, money and IO capacity.

When the read policy is passed in, the clients know immediately what kind of read pattern to expect, rather than infer it as reads take place or from the file suffix.

What changes are included in this PR?

  • Open files with FileSystem.openFile(), passing in file status
  • And read policy of "parquet, vector, random, adaptive"

Are these changes tested?

Through parquet-hadoop.

Are there any user-facing changes?

no.

Closes #3078

* Open files with FileSystem.openFile(), passing in file status
* And read policy of "parquet, vector, random, adaptive"
@steveloughran steveloughran marked this pull request as draft November 27, 2024 12:16
@steveloughran steveloughran marked this pull request as ready for review November 27, 2024 22:05
@wgtmac
Copy link
Member

wgtmac commented Nov 28, 2024

@Fokko @gszadovszky Could you help review this? Thanks!

// 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.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The main issue we hit is that

  • filestatus includes full path
  • hive's wrapper fs has to create new fs instances with the wrapper fs path
  • and s3a was checking the source status matched that of the target path, raising IllegalArgumentException

Hadoop 3.3.5+ only checks filename now

How about

  • log at debug
  • add as suppressed.

Exceptions are too important to lose after all.

Copy link
Contributor

Choose a reason for hiding this comment

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

Sounds good!

@steveloughran
Copy link
Contributor Author

I'm going to propose the following tests to simulate failure and validate handling.

openFile() raises IllegalArgumentException, open() works.

open() succeeds so stream returned.

openFile() raises IllegalArgumentException, open() raises IOE

IOE thrown with IllegalArgumentException as suppressed

openFile() returns a Future, whose get() raises an UncheckedIOE

unchecked IOE unwrapped and thrown.

openFile() returns a stream, open() raises IOE

stream returned, showing open() wasn't invoked.

Does that cover the codepaths?

@gszadovszky
Copy link
Contributor

At the last scenario, the "open() raises IOE" doesn't matter, right? I mean, it can be a test coding detail to throw an exception there so we are sure open() is not invoked, but this is not part of the actual scenario. (We might use Mockito to check if the method was invoked or not, no need for an exception.)

Otherwise, yes, I believe, these are all the scenarios might happen in this code.

@steveloughran
Copy link
Contributor Author

We might use Mockito to check if the method was invoked or not, no need for an exception

Good point, and yes same codepath.

My feelings about mockito are mixed fwiw; tests can be very brittle and often have high maintenance costs. As this is public API stuff, not only shouldn't it be an issue -if it does fail, that's a regression.

The builder API is a real PITA to mock, even though it
works great in application code.

Uses the o.a.h.fs.impl implementation, and I've created
HADOOP-19355. Mark FutureDataInputStreamBuilderImpl as VisibleForTesting
to make it clear that this is allowed.
Copy link
Contributor

@gszadovszky gszadovszky left a comment

Choose a reason for hiding this comment

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

Looks nice and clear. Thanks for the PR!

Copy link
Contributor

@Fokko Fokko left a comment

Choose a reason for hiding this comment

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

LGTM, appreciate the Javadoc/comments, really helps to add Hadoop context

@gszadovszky gszadovszky merged commit f4a3e8b into apache:master Dec 4, 2024
7 checks passed
@steveloughran
Copy link
Contributor Author

shaves a HEAD request!

for s3a it tells things to seek properly rather than having to guess afterwards.

FWIW there's a "whole-file" read policy, we use this in hadoop itself for stuff like distcp...now I need to get the sequential policy into avro so it knows that prefetching is good, rather than a waste of IO capacity

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Use Hadoop FileSystem.openFile() to open files
4 participants