You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by GitBox <gi...@apache.org> on 2022/06/07 11:24:35 UTC

[GitHub] [parquet-mr] steveloughran opened a new pull request, #971: PARQUET-2134: Improve binding to ByteBufferReadable

steveloughran opened a new pull request, #971:
URL: https://github.com/apache/parquet-mr/pull/971

   This extends #951
   
   It improves binding to streams which implement
   ByteBufferReadable through recursive probes of wrapped
   streams and direct querying of the stream on Hadoop 3.3.0+.
   
   Since HDFS-14111 all input streams in the hadoop codebase
   which implement ByteBufferReadable return true on the
   StreamCapabilities probe hasCapability("in:readbytebuffer")
   
   This means the best way to probe for the API on those versions
   is to ask the stream.
   
   The StreamCapabilities probe was added in Hadoop 2.9. Along with
   making all use of `ByteBufferReadable` non-reflective, this makes
   the checks fairly straightforward.
   
   The recursive check is from #951; the change is it no longer
   needs to use reflection.
   
   Tests verify that if a stream implements `ByteBufferReadable' then
   it will be bonded to H2SeekableInputStream, even if multiply wrapped
   by FSDataInputStreams, and that if it doesn't, it won't.
   
   ### Jira
   
   - [X] My PR addresses the following [Parquet Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references them in the PR title. For example, "PARQUET-1234: My Parquet PR"
     - https://issues.apache.org/jira/browse/PARQUET-XXX
     - In case you are adding a dependency, check if the license complies with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for this extremely good reason:
   
   ### Commits
   
   - [X] My commits all reference Jira issues in their subject lines. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [X] In case of new functionality, my PR adds documentation that describes how to use it.
     - All the public functions and the classes in the PR contain Javadoc that explain what it does
   


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] shangxinli commented on pull request #971: PARQUET-2134: Improve binding to ByteBufferReadable

Posted by GitBox <gi...@apache.org>.
shangxinli commented on PR #971:
URL: https://github.com/apache/parquet-mr/pull/971#issuecomment-1193400138

   This PR is combined with https://github.com/apache/parquet-mr/pull/951. 


-- 
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-unsubscribe@parquet.apache.org

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


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

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@parquet.apache.org

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


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

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #971:
URL: https://github.com/apache/parquet-mr/pull/971#discussion_r921361565


##########
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:
   Personally I'm in favor of moving on and adopt the new APIs especially if we are going to depend on Hadoop 3 features more. Maybe we can call the next Parquet release 1.13.0 and declare that it's no longer compatible with older Hadoop versions? 
   
   cc @shangxinli 



-- 
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-unsubscribe@parquet.apache.org

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


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

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@parquet.apache.org

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


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

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] steveloughran closed pull request #971: PARQUET-2134: Improve binding to ByteBufferReadable

Posted by GitBox <gi...@apache.org>.
steveloughran closed pull request #971: PARQUET-2134: Improve binding to ByteBufferReadable
URL: https://github.com/apache/parquet-mr/pull/971


-- 
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-unsubscribe@parquet.apache.org

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


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

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@parquet.apache.org

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


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

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #971:
URL: https://github.com/apache/parquet-mr/pull/971#discussion_r920576934


##########
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 don't understand why Parquet need to use reflection to look up a class defined by itself.



##########
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:
   it appears this is a relatively new method added in https://issues.apache.org/jira/browse/HADOOP-15012 (Hadoop 2.10.0, 2.9.1, 3.1.0 and 3.0.1). Should we care about older provided Hadoop versions?



-- 
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-unsubscribe@parquet.apache.org

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