[GitHub] [parquet-mr] steveloughran commented on a diff in pull request #971: PARQUET-2134: Improve binding to ByteBufferReadable

2022-07-18 Thread GitBox


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


##
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:
   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.



-- 
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



[GitHub] [parquet-mr] steveloughran commented on a diff in pull request #971: PARQUET-2134: Improve binding to ByteBufferReadable

2022-07-14 Thread GitBox


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


##
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:
   if you are targeting the older hadoop releases, you'd also need to build 
java7 artifacts. does anyone want to do that?



-- 
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



[GitHub] [parquet-mr] steveloughran commented on a diff in pull request #971: PARQUET-2134: Improve binding to ByteBufferReadable

2022-07-14 Thread GitBox


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


##
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")) {
+  // 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")

Review Comment:
   I believe it's because of the transitive dependencies; 



-- 
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