You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@orc.apache.org by "Michael Luckey (Jira)" <ji...@apache.org> on 2019/12/27 23:36:00 UTC

[jira] [Commented] (ORC-508) Add a reader/writer that does not depend on Hadoop FileSystem

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

Michael Luckey commented on ORC-508:
------------------------------------

[~omalley] Could you elaborate a bit on your suggestion? While investigating into reading/writing orc files without relying on Hadoop fs/path as [required for apache beam|https://github.com/apache/beam/blob/master/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java#L222-L223] I found  this issue and unfortunately do not quite understand the suggested solution.

IIRC beam had the same issue with ParquetIO which was solved be introducing [some intermediate level of abstraction|https://github.com/apache/parquet-mr/commit/8bfd9b4d8f4fb0a2b522c9328f67eb642066306b] and after a short look it seems possible, that a similar approach could work for an OrcIO as well.

We might prepare a corresponding PR if that seems like a viable solution.

> Add a reader/writer that does not depend on Hadoop FileSystem
> -------------------------------------------------------------
>
>                 Key: ORC-508
>                 URL: https://issues.apache.org/jira/browse/ORC-508
>             Project: ORC
>          Issue Type: Improvement
>          Components: Java
>            Reporter: Ismaël Mejía
>            Priority: Major
>
> It seems that the default implementation classes of Orc today depend on Hadoop FS objects to write. This is not ideal for APIs that do not rely on Hadoop. For some context I was taking a look at adding support for Apache Beam, but Beam's API supports multiple filesystems with a more generic abstraction that relies on Java's Channels and Streams APIs and delegate directly to Distributed FS e.g. Google Cloud Storage, Amazon S3, etc. It would be really nice to have such support in the core implementation and to maybe split the Hadoop dependencies implementation into its own module in the future.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)