Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
steveloughran commented on PR #3079: URL: https://github.com/apache/parquet-java/pull/3079#issuecomment-2517815867 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 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
gszadovszky merged PR #3079: URL: https://github.com/apache/parquet-java/pull/3079 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
steveloughran commented on PR #3079: URL: https://github.com/apache/parquet-java/pull/3079#issuecomment-2511530962 > 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. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
gszadovszky commented on PR #3079: URL: https://github.com/apache/parquet-java/pull/3079#issuecomment-2511466401 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. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
steveloughran commented on PR #3079: URL: https://github.com/apache/parquet-java/pull/3079#issuecomment-2511428665 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? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
gszadovszky commented on code in PR #3079:
URL: https://github.com/apache/parquet-java/pull/3079#discussion_r1865759470
##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopInputFile.java:
##
@@ -70,9 +93,38 @@ public long getLength() {
return stat.getLen();
}
+ /**
+ * Open the file.
+ * 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 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());
Review Comment:
Sounds good!
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
steveloughran commented on code in PR #3079:
URL: https://github.com/apache/parquet-java/pull/3079#discussion_r1865749087
##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopInputFile.java:
##
@@ -70,9 +93,38 @@ public long getLength() {
return stat.getLen();
}
+ /**
+ * Open the file.
+ * 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 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());
Review Comment:
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.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
gszadovszky commented on code in PR #3079:
URL: https://github.com/apache/parquet-java/pull/3079#discussion_r1863833209
##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopInputFile.java:
##
@@ -70,9 +93,38 @@ public long getLength() {
return stat.getLen();
}
+ /**
+ * Open the file.
+ * 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 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());
Review Comment:
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.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
gszadovszky commented on code in PR #3079:
URL: https://github.com/apache/parquet-java/pull/3079#discussion_r1863829269
##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/wrapped/io/FutureIO.java:
##
@@ -70,6 +70,29 @@ public static T awaitFuture(final Future future,
final long timeout, fina
}
}
+ /**
+ * Given a future, evaluate it.
+ *
+ * Any exception generated in the future is
+ * extracted and rethrown.
+ *
+ * @param future future to evaluate
+ * @param 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 awaitFuture(final Future future)
+ throws InterruptedIOException, IOException, RuntimeException {
+try {
+ return future.get();
+} catch (InterruptedException e) {
+ throw (InterruptedIOException) new
InterruptedIOException(e.toString()).initCause(e);
Review Comment:
Oh, missed that, thanks!
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
steveloughran commented on code in PR #3079:
URL: https://github.com/apache/parquet-java/pull/3079#discussion_r1863799758
##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/wrapped/io/FutureIO.java:
##
@@ -70,6 +70,29 @@ public static T awaitFuture(final Future future,
final long timeout, fina
}
}
+ /**
+ * Given a future, evaluate it.
+ *
+ * Any exception generated in the future is
+ * extracted and rethrown.
+ *
+ * @param future future to evaluate
+ * @param 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 awaitFuture(final Future future)
+ throws InterruptedIOException, IOException, RuntimeException {
+try {
+ return future.get();
+} catch (InterruptedException e) {
+ throw (InterruptedIOException) new
InterruptedIOException(e.toString()).initCause(e);
Review Comment:
`initCause()` returns a throwable; you need to casr it back to what the
actual exception is so you can throw it
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
steveloughran commented on code in PR #3079:
URL: https://github.com/apache/parquet-java/pull/3079#discussion_r1863804254
##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopInputFile.java:
##
@@ -70,9 +93,38 @@ public long getLength() {
return stat.getLen();
}
+ /**
+ * Open the file.
+ * 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 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());
Review Comment:
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
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
steveloughran commented on code in PR #3079:
URL: https://github.com/apache/parquet-java/pull/3079#discussion_r1863799758
##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/wrapped/io/FutureIO.java:
##
@@ -70,6 +70,29 @@ public static T awaitFuture(final Future future,
final long timeout, fina
}
}
+ /**
+ * Given a future, evaluate it.
+ *
+ * Any exception generated in the future is
+ * extracted and rethrown.
+ *
+ * @param future future to evaluate
+ * @param 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 awaitFuture(final Future future)
+ throws InterruptedIOException, IOException, RuntimeException {
+try {
+ return future.get();
+} catch (InterruptedException e) {
+ throw (InterruptedIOException) new
InterruptedIOException(e.toString()).initCause(e);
Review Comment:
`initCause()` returns a throwable; you need to case it back to what the
actual exception is
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
gszadovszky commented on code in PR #3079:
URL: https://github.com/apache/parquet-java/pull/3079#discussion_r1861685918
##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/wrapped/io/FutureIO.java:
##
@@ -70,6 +70,29 @@ public static T awaitFuture(final Future future,
final long timeout, fina
}
}
+ /**
+ * Given a future, evaluate it.
+ *
+ * Any exception generated in the future is
+ * extracted and rethrown.
+ *
+ * @param future future to evaluate
+ * @param 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 awaitFuture(final Future future)
+ throws InterruptedIOException, IOException, RuntimeException {
+try {
+ return future.get();
+} catch (InterruptedException e) {
+ throw (InterruptedIOException) new
InterruptedIOException(e.toString()).initCause(e);
Review Comment:
nit: Why the cast?
##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopInputFile.java:
##
@@ -70,9 +93,38 @@ public long getLength() {
return stat.getLen();
}
+ /**
+ * Open the file.
+ * 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 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());
Review Comment:
Shouldn't we at least log the original exception?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
-
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]
Re: [PR] GH-3078: Use Hadoop FileSystem.openFile() to open files [parquet-java]
wgtmac commented on PR #3079: URL: https://github.com/apache/parquet-java/pull/3079#issuecomment-2505167772 @Fokko @gszadovszky Could you help review this? Thanks! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] - To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
