Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -20,14 +20,18 @@
package org.apache.parquet.hadoop.util;

import org.apache.hadoop.fs.ByteBufferReadable;
import org.apache.parquet.io.ParquetDecodingException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
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.lang.reflect.Method;
import java.util.Objects;

/**
Expand All @@ -37,6 +41,39 @@ public class HadoopStreams {

private static final Logger LOG = LoggerFactory.getLogger(HadoopStreams.class);

private static final Class<?> byteBufferReadableClass = getReadableClass();
static final Constructor<SeekableInputStream> h2SeekableConstructor = getH2SeekableConstructor();
Copy link
Contributor

Choose a reason for hiding this comment

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

Rather than using two static methods, you can use DynConstructors instead to make this one expression and reduce error handling code:

private static final DynConstructors.Ctor<SeekableInputStream> h2streamCtor =
    DynConstructors.Builder(SeekableInputStream.class)
    .impl("org.apache.parquet.hadoop.util.H2SeekableInputStream", FSDataInputStream.class)
    .orNull()
    .build()

...
if (h2streamCtor != null) {
  return h2streamCtor.newInstance(stream);
}

Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

Huh, I guess this was how it was before? Nevermind on the refactoring then.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, I copied most from the old code to avoid refactoring. I think we can greatly simplify it because it was still taking Hadoop1 into account. We still have to check if the wrapped stream is ByteBufferReadable: https://github.com/apache/hadoop/blob/release-2.4.1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java#L142-L148

The `hasCapabilities does the same but in a more elegant way.


private static Class<?> getReadableClass() {
try {
return Class.forName("org.apache.hadoop.fs.ByteBufferReadable");
} catch (ClassNotFoundException | NoClassDefFoundError e) {
return null;
}
}

@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;
}
}

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 FSDataInputStream} in a {@link SeekableInputStream}
* implementation for Parquet readers.
Expand All @@ -46,41 +83,91 @@ public class HadoopStreams {
*/
public static SeekableInputStream wrap(FSDataInputStream stream) {
Objects.requireNonNull(stream, "Cannot wrap a null input stream");
if (isWrappedStreamByteBufferReadable(stream)) {
return new H2SeekableInputStream(stream);
} else {
return new H1SeekableInputStream(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 (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);
} catch (InvocationTargetException e) {
throw new ParquetDecodingException(
"Could not instantiate H2SeekableInputStream", e.getTargetException());
}
}
return new H1SeekableInputStream(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
*/
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;
private static Boolean isWrappedStreamByteBufferReadableHasCapabilities(FSDataInputStream stream) {
Method methodHasCapabilities;
try {
methodHasCapabilities = stream.getClass().getMethod("hasCapability", String.class);
} catch (Exception e) {
return null;
}
try {
if ((Boolean) methodHasCapabilities.invoke(stream, "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;
}
} catch (IllegalAccessException | InvocationTargetException 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));
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