[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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)

Reply via email to