Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -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;

Expand All @@ -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.
Expand All @@ -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) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Usually, isSomething methods return a boolean. This is unwrapping, so I'd prefer naming it unwrapByteBufferReadableLegacy or something to be more clear.

InputStream wrapped = stream.getWrappedStream();
if (wrapped instanceof FSDataInputStream) {
LOG.debug("Checking on wrapped stream {} of {} whether is ByteBufferReadable", wrapped, stream);
return isWrappedStreamByteBufferReadableLegacy(((FSDataInputStream) wrapped));
Copy link
Contributor

Choose a reason for hiding this comment

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

Why would a FSDataInputStream have another one inside?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This came from the issue from Presto: prestodb/presto#17435

Copy link
Contributor

Choose a reason for hiding this comment

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

you can if you try hard, it's just really unusual

you can never wrap an instance by itself.

}
if (stream.getWrappedStream() instanceof ByteBufferReadable) {
Copy link
Contributor

Choose a reason for hiding this comment

The 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);
Expand All @@ -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) {
if (hasCapabilitiesMethod.isNoop()) {
// When the method is not available, just return a null
return null;
}

Boolean hasCapabilities = hasCapabilitiesMethod.invoke(stream, "in:readbytebuffer");
Copy link
Contributor

Choose a reason for hiding this comment

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

Is this a boxed boolean? If so, should we update the check to handle null?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I was assuming that it needed to be an object, but a primitive works as well, so changed that. Thanks for catching this!


if (hasCapabilities) {
Copy link
Contributor

Choose a reason for hiding this comment

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

This variable would be more clear if it were called isByteBufferReadable since that's the capability we are checking for.

// stream is issuing the guarantee that it implements the
// API. Holds for all implementations in hadoop-*
// since Hadoop 3.3.0 (HDFS-14111).
Expand All @@ -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;
}
Expand Down
2 changes: 1 addition & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -592,7 +592,7 @@
<profile>
<id>hadoop2</id>
<properties>
<hadoop.version>2.9.2</hadoop.version>
<hadoop.version>2.7.3</hadoop.version>
</properties>
</profile>

Expand Down