rdblue commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1183059127


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##########
@@ -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) {
+    InputStream wrapped = stream.getWrappedStream();
+    if (wrapped instanceof FSDataInputStream) {
+      LOG.debug("Checking on wrapped stream {} of {} whether is 
ByteBufferReadable", wrapped, stream);
+      return isWrappedStreamByteBufferReadableLegacy(((FSDataInputStream) 
wrapped));

Review Comment:
   Why would a FSDataInputStream have another one inside?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##########
@@ -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) {
+    InputStream wrapped = stream.getWrappedStream();
+    if (wrapped instanceof FSDataInputStream) {
+      LOG.debug("Checking on wrapped stream {} of {} whether is 
ByteBufferReadable", wrapped, stream);
+      return isWrappedStreamByteBufferReadableLegacy(((FSDataInputStream) 
wrapped));
+    }
+    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;

Review Comment:
   Why not return false?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to