-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-2134: Fix type checking in HadoopStreams.wrap #951
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,16 +19,15 @@ | |
|
|
||
| package org.apache.parquet.hadoop.util; | ||
|
|
||
| import org.apache.hadoop.fs.ByteBufferReadable; | ||
| import org.apache.hadoop.fs.FSDataInputStream; | ||
| import org.apache.hadoop.fs.FSDataOutputStream; | ||
| import org.apache.parquet.io.ParquetDecodingException; | ||
| import org.apache.parquet.io.SeekableInputStream; | ||
| import org.apache.parquet.io.PositionOutputStream; | ||
| import org.apache.parquet.io.SeekableInputStream; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.lang.reflect.Constructor; | ||
| import java.lang.reflect.InvocationTargetException; | ||
| import java.io.InputStream; | ||
| import java.util.Objects; | ||
|
|
||
| /** | ||
|
|
@@ -38,9 +37,6 @@ public class HadoopStreams { | |
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(HadoopStreams.class); | ||
|
|
||
| private static final Class<?> byteBufferReadableClass = getReadableClass(); | ||
| static final Constructor<SeekableInputStream> h2SeekableConstructor = getH2SeekableConstructor(); | ||
|
|
||
| /** | ||
| * Wraps a {@link FSDataInputStream} in a {@link SeekableInputStream} | ||
| * implementation for Parquet readers. | ||
|
|
@@ -50,51 +46,45 @@ public class HadoopStreams { | |
| */ | ||
| public static SeekableInputStream wrap(FSDataInputStream stream) { | ||
| Objects.requireNonNull(stream, "Cannot wrap a null input stream"); | ||
| if (byteBufferReadableClass != null && h2SeekableConstructor != null && | ||
| byteBufferReadableClass.isInstance(stream.getWrappedStream())) { | ||
| try { | ||
| return h2SeekableConstructor.newInstance(stream); | ||
| } catch (InstantiationException | IllegalAccessException e) { | ||
| LOG.warn("Could not instantiate H2SeekableInputStream, falling back to byte array reads", e); | ||
| return new H1SeekableInputStream(stream); | ||
| } catch (InvocationTargetException e) { | ||
| throw new ParquetDecodingException( | ||
| "Could not instantiate H2SeekableInputStream", e.getTargetException()); | ||
| } | ||
| if (isWrappedStreamByteBufferReadable(stream)) { | ||
| return new H2SeekableInputStream(stream); | ||
| } else { | ||
| return new H1SeekableInputStream(stream); | ||
| } | ||
| } | ||
|
|
||
| private static Class<?> getReadableClass() { | ||
| try { | ||
| return Class.forName("org.apache.hadoop.fs.ByteBufferReadable"); | ||
| } catch (ClassNotFoundException | NoClassDefFoundError e) { | ||
| return null; | ||
| /** | ||
| * Is the inner stream byte buffer readable? | ||
| * The test is "the stream is not FSDataInputStream | ||
| * and implements ByteBufferReadable' | ||
| * | ||
| * That is: all streams which implement ByteBufferReadable | ||
| * other than FSDataInputStream successfuly support read(ByteBuffer). | ||
| * This is true for all filesytem clients the hadoop codebase. | ||
| * | ||
| * In hadoop 3.3.0+, the StreamCapabilities probe can be used to | ||
| * check this: only those streams which provide the read(ByteBuffer) | ||
| * semantics MAY return true for the probe "in:readbytebuffer"; | ||
| * FSDataInputStream will pass the probe down to the underlying stream. | ||
| * | ||
| * @param stream stream to probe | ||
| * @return true if it is safe to a H2SeekableInputStream to access the data | ||
| */ | ||
| private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream stream) { | ||
| if (stream.hasCapability("in:readbytebuffer")) { | ||
| // stream is issuing the guarantee that it implements the | ||
| // API. Holds for all implementations in hadoop-* | ||
| // since Hadoop 3.3.0 (HDFS-14111). | ||
| return true; | ||
| } | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| private static Class<SeekableInputStream> getH2SeekableClass() { | ||
| try { | ||
| return (Class<SeekableInputStream>) Class.forName( | ||
| "org.apache.parquet.hadoop.util.H2SeekableInputStream"); | ||
| } catch (ClassNotFoundException | NoClassDefFoundError e) { | ||
| return null; | ||
| InputStream wrapped = stream.getWrappedStream(); | ||
| if (wrapped instanceof FSDataInputStream) { | ||
| LOG.debug("Checking on wrapped stream {} of {} whether is ByteBufferReadable", wrapped, stream); | ||
| return isWrappedStreamByteBufferReadable(((FSDataInputStream) wrapped)); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is there a corner case that can cause an infinite loop?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, it could be. But it may be hard to create such a case. As its code shows, FSDataInputStream is a wrapper class of an inputstream. When we check the wrapped inputstream recursively, it would finally reach an inputstream whose type is not FSDataInputStream. A developer could override
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I understand it would be very rare case but once that happen it would be hard to debug this 'hang' issue. Let's do two things: 1) Add check if it is 'this'; Throw exception if that happens; 2) Add debug log; When it hangs, developer can enable debug log and see what parquet-mr is doing.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good suggestions! I have updated as the comment. |
||
| } | ||
| return wrapped instanceof ByteBufferReadable; | ||
| } | ||
|
|
||
| private static Constructor<SeekableInputStream> getH2SeekableConstructor() { | ||
| Class<SeekableInputStream> h2SeekableClass = getH2SeekableClass(); | ||
| if (h2SeekableClass != null) { | ||
| try { | ||
| return h2SeekableClass.getConstructor(FSDataInputStream.class); | ||
| } catch (NoSuchMethodException e) { | ||
| return null; | ||
| } | ||
| } | ||
| return null; | ||
| } | ||
|
|
||
| /** | ||
| * Wraps a {@link FSDataOutputStream} in a {@link PositionOutputStream} | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We don't have the Hadoop 3..3.0 yet in Parquet. Does it mean we need to hold of this PR?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
no, the StreamCapabilities probe has been around since hadoop 2. it is just in 3.3.0 all streams which implement the api return true for this probe...a probe which gets passed down the wrapped streams. It avoids looking at the wrapped streams as you should be able to trust the response (put differently: if something lied it is in trouble)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@steveloughran @shangxinli it looks like the API is not available in Hadoop 2.8.x, so it will create issues for projects that want to use the latest version of Parquet but still want to keep Hadoop 2.8.x.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
also see related JIRA: https://issues.apache.org/jira/browse/PARQUET-2276