-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-2276: Bring back support for Hadoop 2.7.3 #1084
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 2 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 |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ | |
| import org.apache.hadoop.fs.FSDataOutputStream; | ||
| import org.apache.parquet.io.PositionOutputStream; | ||
| import org.apache.parquet.io.SeekableInputStream; | ||
| import org.apache.parquet.util.DynMethods; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -37,6 +38,13 @@ public class HadoopStreams { | |
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(HadoopStreams.class); | ||
|
|
||
| private static final DynMethods.UnboundMethod hasCapabilitiesMethod = | ||
| new DynMethods | ||
| .Builder("hasCapabilities") | ||
| .impl(FSDataInputStream.class, "hasCapabilities", String.class) | ||
| .orNoop() | ||
| .build(); | ||
|
|
||
| /** | ||
| * Wraps a {@link FSDataInputStream} in a {@link SeekableInputStream} | ||
| * implementation for Parquet readers. | ||
|
|
@@ -46,7 +54,39 @@ public class HadoopStreams { | |
| */ | ||
| public static SeekableInputStream wrap(FSDataInputStream stream) { | ||
| Objects.requireNonNull(stream, "Cannot wrap a null input stream"); | ||
| if (isWrappedStreamByteBufferReadable(stream)) { | ||
|
|
||
| // Try to check using hasCapabilities(str) | ||
| Boolean hasCapabilitiesResult = isWrappedStreamByteBufferReadableHasCapabilities(stream); | ||
|
|
||
| // If it is null, then fall back to the old method | ||
| if (hasCapabilitiesResult != null) { | ||
| if (hasCapabilitiesResult) { | ||
| return new H2SeekableInputStream(stream); | ||
| } else { | ||
| return new H1SeekableInputStream(stream); | ||
| } | ||
| } | ||
|
|
||
| return isWrappedStreamByteBufferReadableLegacy(stream); | ||
| } | ||
|
|
||
| /** | ||
| * Is the inner stream byte buffer readable? | ||
| * The test is 'the stream is not FSDataInputStream | ||
| * and implements ByteBufferReadable' | ||
| * | ||
| * This logic is only used for Hadoop <2.9.x, and <3.x.x | ||
| * | ||
| * @param stream stream to probe | ||
| * @return A H2SeekableInputStream to access, or H1SeekableInputStream if the stream is not seekable | ||
| */ | ||
| private static SeekableInputStream isWrappedStreamByteBufferReadableLegacy(FSDataInputStream stream) { | ||
| InputStream wrapped = stream.getWrappedStream(); | ||
| if (wrapped instanceof FSDataInputStream) { | ||
| LOG.debug("Checking on wrapped stream {} of {} whether is ByteBufferReadable", wrapped, stream); | ||
Fokko marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| return isWrappedStreamByteBufferReadableLegacy(((FSDataInputStream) wrapped)); | ||
|
||
| } | ||
| if (stream.getWrappedStream() instanceof ByteBufferReadable) { | ||
|
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 prefer using the same whitespace conventions as in Iceberg, although that's a bit more relaxed over here. |
||
| return new H2SeekableInputStream(stream); | ||
| } else { | ||
| return new H1SeekableInputStream(stream); | ||
|
|
@@ -55,23 +95,31 @@ public static SeekableInputStream wrap(FSDataInputStream stream) { | |
|
|
||
| /** | ||
| * Is the inner stream byte buffer readable? | ||
| * The test is "the stream is not FSDataInputStream | ||
| * 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. | ||
| * other than FSDataInputStream successfully support read(ByteBuffer). | ||
| * This is true for all filesystem 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 | ||
| * @return true if it is safe to a H2SeekableInputStream to access | ||
| * the data, null when it cannot be determined because of missing hasCapabilities | ||
| */ | ||
| private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream stream) { | ||
| if (stream.hasCapability("in:readbytebuffer")) { | ||
| private static Boolean isWrappedStreamByteBufferReadableHasCapabilities(FSDataInputStream stream) { | ||
Fokko marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| if (hasCapabilitiesMethod.isNoop()) { | ||
| // When the method is not available, just return a null | ||
| return null; | ||
| } | ||
|
|
||
| Boolean hasCapabilities = hasCapabilitiesMethod.invoke(stream, "in:readbytebuffer"); | ||
|
||
|
|
||
| if (hasCapabilities) { | ||
|
||
| // stream is issuing the guarantee that it implements the | ||
| // API. Holds for all implementations in hadoop-* | ||
| // since Hadoop 3.3.0 (HDFS-14111). | ||
|
|
@@ -80,7 +128,7 @@ private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream strea | |
| InputStream wrapped = stream.getWrappedStream(); | ||
| if (wrapped instanceof FSDataInputStream) { | ||
| LOG.debug("Checking on wrapped stream {} of {} whether is ByteBufferReadable", wrapped, stream); | ||
| return isWrappedStreamByteBufferReadable(((FSDataInputStream) wrapped)); | ||
| return isWrappedStreamByteBufferReadableHasCapabilities(((FSDataInputStream) wrapped)); | ||
| } | ||
| return wrapped instanceof ByteBufferReadable; | ||
| } | ||
|
|
||
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.
Usually,
isSomethingmethods return a boolean. This is unwrapping, so I'd prefer naming itunwrapByteBufferReadableLegacyor something to be more clear.