Skip to content
Closed
Show file tree
Hide file tree
Changes from all 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 @@ -84,7 +84,7 @@ public int read(ByteBuffer buf) throws IOException {
}

public static void readFully(Reader reader, ByteBuffer buf) throws IOException {
// unfortunately the Hadoop APIs seem to not have a 'readFully' equivalent for the byteBuffer read
// unfortunately the Hadoop 2 APIs do not have a 'readFully' equivalent for the byteBuffer read
// calls. The read(ByteBuffer) call might read fewer than byteBuffer.hasRemaining() bytes. Thus we
// have to loop to ensure we read them.
while (buf.hasRemaining()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

/**
Expand All @@ -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.
Expand All @@ -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")) {
Copy link
Member

Choose a reason for hiding this comment

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

it appears this is a relatively new method added in https://issues.apache.org/jira/browse/HADOOP-15012 (Hadoop 2.10.0, 2.9.1, 3.1.0 and 3.0.1). Should we care about older provided Hadoop versions?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

if you are targeting the older hadoop releases, you'd also need to build java7 artifacts. does anyone want to do that?

Copy link
Member

Choose a reason for hiding this comment

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

Personally I'm in favor of moving on and adopt the new APIs especially if we are going to depend on Hadoop 3 features more. Maybe we can call the next Parquet release 1.13.0 and declare that it's no longer compatible with older Hadoop versions?

cc @shangxinli

Copy link
Contributor Author

Choose a reason for hiding this comment

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

that would be nice. do that and the library we are doing to help give 3.2+ apps access to the higher performance cloud storage APIs when available would be great.

Copy link
Contributor

Choose a reason for hiding this comment

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

Let's be careful about introducing incompatibility & Hadoop is a fundamental dependency for Parquet.

// 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")
Copy link
Member

Choose a reason for hiding this comment

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

I don't understand why Parquet need to use reflection to look up a class defined by itself.

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 believe it's because of the transitive dependencies;

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));
}
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}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,18 @@

package org.apache.parquet.hadoop.util;

import org.apache.hadoop.fs.ByteBufferReadable;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.parquet.hadoop.TestUtils;
import org.apache.parquet.io.SeekableInputStream;
import org.junit.Assert;
import org.junit.Test;
import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.Callable;

import static org.apache.parquet.hadoop.util.HadoopStreams.wrap;
import static org.apache.parquet.hadoop.util.MockHadoopInputStream.TEST_ARRAY;

public class TestHadoop2ByteBufferReads {
Expand Down Expand Up @@ -396,4 +399,48 @@ public void testDirectReadFullyPositionAndLimit() throws Exception {
Assert.assertEquals("Buffer contents should match",
ByteBuffer.wrap(TEST_ARRAY, 0, 7), readBuffer);
}

@Test
public void testCreateStreamNoByteBufferReadable() {
final SeekableInputStream s = wrap(new FSDataInputStream(
new MockHadoopInputStream()));
Assert.assertTrue("Wrong wrapper: " + s,
s instanceof H1SeekableInputStream);
}

@Test
public void testDoubleWrapNoByteBufferReadable() {
final SeekableInputStream s = wrap(new FSDataInputStream(
new FSDataInputStream(new MockHadoopInputStream())));
Assert.assertTrue("Wrong wrapper: " + s,
s instanceof H1SeekableInputStream);
}

@Test
public void testCreateStreamWithByteBufferReadable() {
final SeekableInputStream s = wrap(new FSDataInputStream(
new MockByteBufferInputStream()));
Assert.assertTrue("Wrong wrapper: " + s,
s instanceof H2SeekableInputStream);
}

@Test
public void testDoubleWrapByteBufferReadable() {
final SeekableInputStream s = wrap(new FSDataInputStream(
new FSDataInputStream(new MockByteBufferInputStream())));
Assert.assertTrue("Wrong wrapper: " + s,
s instanceof H2SeekableInputStream);
}

/**
* Input stream which claims to implement ByteBufferReadable.
*/
private static final class MockByteBufferInputStream
extends MockHadoopInputStream implements ByteBufferReadable {

@Override
public int read(final ByteBuffer buf) {
return 0;
}
}
}