[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2023-04-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17712127#comment-17712127
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

sunchao commented on code in PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#discussion_r1166131285


##
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:
   also see related JIRA: https://issues.apache.org/jira/browse/PARQUET-2276





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
> Fix For: 1.13.0
>
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2023-04-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17712118#comment-17712118
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

sunchao commented on code in PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#discussion_r1166090906


##
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:
   @steveloughran @shangxinli it looks like the API is not available in Hadoop 
2.8.x, so it will create issues for projects that want to use the latest 
version of Parquet but still want to keep Hadoop 2.8.x.





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
> Fix For: 1.13.0
>
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-25 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17570831#comment-17570831
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

steveloughran closed pull request #971: PARQUET-2134: Improve binding to 
ByteBufferReadable
URL: https://github.com/apache/parquet-mr/pull/971




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-24 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17570552#comment-17570552
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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. 




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-24 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17570538#comment-17570538
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

shangxinli commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1193383006

   LGTM




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-24 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17570539#comment-17570539
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

shangxinli merged PR #951:
URL: https://github.com/apache/parquet-mr/pull/951




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-19 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17568609#comment-17568609
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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. 





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-18 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17568138#comment-17568138
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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.





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17566915#comment-17566915
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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 





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17566826#comment-17566826
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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?





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17566825#comment-17566825
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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; 





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17566550#comment-17566550
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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 

[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-12 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17565427#comment-17565427
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

steveloughran commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1181629067

   thanks. created 
[HADOOP-18336](https://issues.apache.org/jira/browse/HADOOP-18336)
   tag FSDataInputStream.getWrappedStream() @Public/@Stable  to make sure that 
hadoop code knows external libs may be calling the method.




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17562229#comment-17562229
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

7c00 commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1173982766

   @shangxinli Thank you for reminding me. I have squashed the PR and added 
@steveloughran as the co-author.




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-07-02 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17561766#comment-17561766
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

shangxinli commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1172924045

   @7c00 and @steveloughran Thank both of you for the great contribution! This 
PR comes from two authors. Can @7c00 add @steveloughran as the co-author to 
this PR? 
[This](https://github.blog/2018-01-29-commit-together-with-co-authors/) is an 
example. 




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17556465#comment-17556465
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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


##
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:
   no, the StreamCapabilities probe has been around since hadoop 2. it is just 
in 3.3.0 all streams which implement the api return true for this probe...a 
probe which gets passed down the wrapped streams. It avoids looking at the 
wrapped streams as you should be able to trust the response (put differently: 
if something lied it is in trouble)





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17555809#comment-17555809
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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


##
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:
   We don't have the Hadoop 3..3.0 yet in Parquet. Does it mean we need to hold 
of this PR? 





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17553563#comment-17553563
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

steveloughran commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1153871066

   whoever actually commits this can use the github squash option to combine 
all commits into one before merging.
   
   FYI, I've just started writing a shim library so that apps compiling against 
hadoop 3.2.0 wil be able to invoke the 3.3+ API calls when present: 
[HADOOP-18287](https://issues.apache.org/jira/browse/HADOOP-18287). 
   
   First parquet will need to be able to compile/link against hadoop 3.x: #976 
976




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17553401#comment-17553401
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

7c00 commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1153522439

   Thanks @steveloughran @shangxinli . I have cherry-picked the commit from 
https://github.com/apache/parquet-mr/pull/971




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-09 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17552281#comment-17552281
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

shangxinli commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1151315837

   > I've taken this PR and added the changes I was suggesting, plus tests. see 
#971. If you take that extra commit and merge it in here, it should complete 
this PR
   
   @7c00 Are you OK since you are originally created this PR? 




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-07 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17550944#comment-17550944
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

steveloughran commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1148538544

   I've taken this PR and added the changes I was suggesting, plus tests. see 
#971. If you take that extra commit and merge it in here, it should complete 
this PR




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-07 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17550942#comment-17550942
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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
   




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-06 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17550597#comment-17550597
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -66,6 +67,19 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 }
   }
 
+  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped == stream) {
+  throw new ParquetDecodingException("Illegal FSDataInputStream as wrapped 
itself");

Review Comment:
   this can't happen. the inner stream is set in the constructor, so cannot 
take the not-yet-constructed class as an argument...no need to worry about 
recursion.





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-06 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17550595#comment-17550595
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -51,7 +52,7 @@ 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())) {
+isWrappedStreamByteBufferReadable(stream)) {

Review Comment:
   this is really going into the internals of the hadoop classes and 
potentially tricky if there is any dynamic decision making in the inner class. 
The good news there is I don't see anything doing that.
   
   there is a way to ask (hadoop 3.2+) if a stream does support the API before 
calling, using the StreamCapabilities interface.
   https://issues.apache.org/jira/browse/HDFS-14111
   
   ```java
   if (stream.hasCapability( "in:readbytebuffer") {
 // stream is confident it has the api
   ) else {
 // do the checking of the inner class
   }



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -66,6 +67,15 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 }
   }
 
+  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  return isWrappedStreamByteBufferReadable(((FSDataInputStream) wrapped));

Review Comment:
   you can't. the inner stream is set in the constructor, so cannot take the 
not-yet-constructed class as an argument.





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-06 Thread Steve Loughran (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17550592#comment-17550592
 ] 

Steve Loughran commented on PARQUET-2134:
-

have you a full stack trace. as IMO the issue isn't type checking, it is 
handling streams to filesystem clients which don't support the 
ByteBufferReadable interface

> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-05-11 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17534953#comment-17534953
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

7c00 commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1123913274

   > @7c00 Do you have time to look into the last feedback?
   
   @shangxinli Thanks for your comments. Sorry for late to reply. I think it's 
ok to add some unit tests, and I am going to do that in this weekend. I will 
ping you when finished. Thank you in advance.




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-05-11 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17534948#comment-17534948
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

shangxinli commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1123901314

   @7c00 Do you have time to look into the last feedback? 




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-04-03 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17516600#comment-17516600
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

shangxinli commented on PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1086966074

   Thanks for adding the check and debug log. LGTM! One more thing(sorry for 
not asking at first-round review), do you think it makes sense to add tests?




> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-03-23 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17511570#comment-17511570
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

7c00 commented on a change in pull request #951:
URL: https://github.com/apache/parquet-mr/pull/951#discussion_r833885795



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
##
@@ -66,6 +67,15 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 }
   }
 
+  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  return isWrappedStreamByteBufferReadable(((FSDataInputStream) wrapped));

Review comment:
   Good suggestions! I have updated as the comment.




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


> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-03-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17509496#comment-17509496
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

shangxinli commented on a change in pull request #951:
URL: https://github.com/apache/parquet-mr/pull/951#discussion_r830648757



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
##
@@ -66,6 +67,15 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 }
   }
 
+  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  return isWrappedStreamByteBufferReadable(((FSDataInputStream) wrapped));

Review comment:
   I understand it would be very rare case but once that happen it would be 
hard to debug this 'hang' issue. Let's do two things: 1) Add check if it is 
'this'; Throw exception if that happens; 2) Add debug log; When it hangs, 
developer can enable debug log and see what parquet-mr is doing. 




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


> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-03-16 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17507386#comment-17507386
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

7c00 commented on a change in pull request #951:
URL: https://github.com/apache/parquet-mr/pull/951#discussion_r827693142



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
##
@@ -66,6 +67,15 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 }
   }
 
+  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  return isWrappedStreamByteBufferReadable(((FSDataInputStream) wrapped));

Review comment:
   Yes, it could be. But it may be hard to create such a case. As its code 
shows, FSDataInputStream is a wrapper class of an inputstream. When we check 
the wrapped inputstream recursively, it would finally reach an inputstream 
whose type is not FSDataInputStream. A developer could override 
`getWrappedStream` as `return this` to cause an infinite loop, while this makes 
no sense.




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


> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-03-16 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17507385#comment-17507385
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

7c00 commented on a change in pull request #951:
URL: https://github.com/apache/parquet-mr/pull/951#discussion_r827693142



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
##
@@ -66,6 +67,15 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 }
   }
 
+  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  return isWrappedStreamByteBufferReadable(((FSDataInputStream) wrapped));

Review comment:
   Yes, it could be. But it may be hard to create such a case. As its code 
shows, FSDataInputStream is a wrapper class of an inputstream. When we check 
the wrapped inputstream recursively, it would finally reach an inputstream 
whose type is not FSDataInputStream. A developer could override 
`getWrappedStream` as `return this` to cause an infinite loop, while this makes 
no sence.




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


> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-03-16 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17507384#comment-17507384
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

7c00 commented on a change in pull request #951:
URL: https://github.com/apache/parquet-mr/pull/951#discussion_r827693142



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
##
@@ -66,6 +67,15 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 }
   }
 
+  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  return isWrappedStreamByteBufferReadable(((FSDataInputStream) wrapped));

Review comment:
   Yes, it could be. But it may be hard to create such a case. As its code 
shows, FSDataInputStream is a wrapper class of an inputstream. When we check 
the wrapped inputstream recursively, it would finally reach an inputstream 
whose type is not FSDataInputStream. A developer could override 
`getWrappedStream` as `return this`, while this makes no sence.




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


> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-03-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17506342#comment-17506342
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

shangxinli commented on a change in pull request #951:
URL: https://github.com/apache/parquet-mr/pull/951#discussion_r826168694



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
##
@@ -66,6 +67,15 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 }
   }
 
+  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  return isWrappedStreamByteBufferReadable(((FSDataInputStream) wrapped));

Review comment:
   Is there a corner case that can cause an infinite loop?




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


> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-03-08 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17503301#comment-17503301
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

7c00 commented on pull request #951:
URL: https://github.com/apache/parquet-mr/pull/951#issuecomment-1062551863


   Related issue: https://github.com/prestodb/presto/pull/17435


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


> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-03-08 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17503284#comment-17503284
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

7c00 opened a new pull request #951:
URL: https://github.com/apache/parquet-mr/pull/951


   Make sure you have checked _all_ steps below.
   
   ### 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-2134
 - 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
   
   - [x] does not need testing for this extremely good reason:
   
   ### Commits
   
   - [ ] 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-unsubscr...@parquet.apache.org

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


> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)