You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Sean Mackrory (JIRA)" <ji...@apache.org> on 2018/08/22 14:24:00 UTC

[jira] [Commented] (HADOOP-15688) ABFS: InputStream wrapped in FSDataInputStream twice

    [ https://issues.apache.org/jira/browse/HADOOP-15688?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16588910#comment-16588910 ] 

Sean Mackrory commented on HADOOP-15688:
----------------------------------------

If anyone's wondering, here's the Parquet code that trips up over this. It gets FSDataInputStream(FSDataInputStream(InputStream)) so FSDataInputStream.getWrappedStream() returns another FSDataInputStream which of course does expose the ByteBufferReadable interface. So it proceeds to set up a reader that uses ByteBuffer reads and by the time it actually attempts a read the inner FSDataInputStream checks the inner-most InputStream and sees that it does NOT implement the ByteBufferReadable interface and throws an OperationNotSupportedException, at which point Parquet doesn't know what to do so it just throws the exception up to the top.

https://github.com/apache/parquet-mr/blob/8bfd9b4d8f4fb0a2b522c9328f67eb642066306b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L54

> ABFS: InputStream wrapped in FSDataInputStream twice
> ----------------------------------------------------
>
>                 Key: HADOOP-15688
>                 URL: https://issues.apache.org/jira/browse/HADOOP-15688
>             Project: Hadoop Common
>          Issue Type: Sub-task
>            Reporter: Sean Mackrory
>            Assignee: Sean Mackrory
>            Priority: Major
>         Attachments: HADOOP-15688.001.patch
>
>
> I can't read Parquet files from ABFS. It has 2 different implementations to read seekable streams, and it'll use the one that uses ByteBuffer reads if it can. It currently decides to use the ByteBuffer read implementation because the FSDataInputStream it gets back wraps another FSDataInputStream, which implements ByteBufferReadable.
> That's not the most robust way to check that ByteBufferReads are supported by the ultimately underlying InputStream, but it's unnecessary and probably a mistake to double-wrap the InputStream, so let's not.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org