This is an automated email from the ASF dual-hosted git repository.

gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-java.git


The following commit(s) were added to refs/heads/master by this push:
     new e3257968d GH-3239: Improve ByteBufferReadable detection in 
HadoopStream (#3259)
e3257968d is described below

commit e3257968d889470d0cb52c393b9e1bb144a2f121
Author: Yuming Wang <[email protected]>
AuthorDate: Thu Jul 31 10:00:28 2025 +0800

    GH-3239: Improve ByteBufferReadable detection in HadoopStream (#3259)
---
 .../java/org/apache/parquet/hadoop/util/HadoopStreams.java  | 13 +------------
 1 file changed, 1 insertion(+), 12 deletions(-)

diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
index eebc59987..fc84729c7 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
@@ -27,7 +27,6 @@ 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.apache.parquet.util.DynMethods;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,11 +37,6 @@ 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.
@@ -111,12 +105,7 @@ public class HadoopStreams {
    * the data, null when it cannot be determined because of missing 
hasCapabilities
    */
   private static Boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
-    if (hasCapabilitiesMethod.isNoop()) {
-      // When the method is not available, just return a null
-      return null;
-    }
-
-    boolean isByteBufferReadable = hasCapabilitiesMethod.invoke(stream, 
"in:readbytebuffer");
+    boolean isByteBufferReadable = stream.hasCapability("in:readbytebuffer");
 
     if (isByteBufferReadable) {
       // stream is issuing the guarantee that it implements the

Reply via email to