Skip to content

Commit e325796

Browse files
authored
GH-3239: Improve ByteBufferReadable detection in HadoopStream (#3259)
1 parent 0a86015 commit e325796

File tree

1 file changed

+1
-12
lines changed

1 file changed

+1
-12
lines changed

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java

Lines changed: 1 addition & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@
2727
import org.apache.hadoop.fs.FSDataOutputStream;
2828
import org.apache.parquet.io.PositionOutputStream;
2929
import org.apache.parquet.io.SeekableInputStream;
30-
import org.apache.parquet.util.DynMethods;
3130
import org.slf4j.Logger;
3231
import org.slf4j.LoggerFactory;
3332

@@ -38,11 +37,6 @@ public class HadoopStreams {
3837

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

41-
private static final DynMethods.UnboundMethod hasCapabilitiesMethod = new DynMethods.Builder("hasCapabilities")
42-
.impl(FSDataInputStream.class, "hasCapabilities", String.class)
43-
.orNoop()
44-
.build();
45-
4640
/**
4741
* Wraps a {@link FSDataInputStream} in a {@link SeekableInputStream}
4842
* implementation for Parquet readers.
@@ -111,12 +105,7 @@ private static Function<FSDataInputStream, SeekableInputStream> unwrapByteBuffer
111105
* the data, null when it cannot be determined because of missing hasCapabilities
112106
*/
113107
private static Boolean isWrappedStreamByteBufferReadable(FSDataInputStream stream) {
114-
if (hasCapabilitiesMethod.isNoop()) {
115-
// When the method is not available, just return a null
116-
return null;
117-
}
118-
119-
boolean isByteBufferReadable = hasCapabilitiesMethod.invoke(stream, "in:readbytebuffer");
108+
boolean isByteBufferReadable = stream.hasCapability("in:readbytebuffer");
120109

121110
if (isByteBufferReadable) {
122111
// stream is issuing the guarantee that it implements the

0 commit comments

Comments
 (0)