shangxinli commented on code in PR #971:
URL: https://github.com/apache/parquet-mr/pull/971#discussion_r924651838


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##########
@@ -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")) {

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



-- 
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: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to