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

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

Thanks, [~omalley] for your explanations. If I understood correctly you suggest 
creating some overlapping classes which could be used instead dependency on 
common Hadoop classes. But actually adding Hadoop jars to a potential OrcIOs 
class path is not the main difficulty on Apache Beam. Although it would be 
preferable to have the least possible set of dependencies (if just to prevent 
package sizes and dependency hell) the issue is more about the interface Beam 
exposes to Ios, which is based on channels/stream, not necessarily something  
path-alike, which makes it not ideal to implement. This would not only helpful 
for Apache Beam, but might also be helpful for others not working on Hadoop FS 
based systems. 

Of course we totally agree on not breaking backwards compatibility. Without 
having fully implemented something here, I think we could (on beam side) wrap 
streams into FSData(In|Out)putStreams, mock Path (which seems apart from 
accessing FileSystem mainly be used for error messages) and FileSystem.create. 
But this seems rather odd and potentially gives bad user experience as we need 
to tightly couple to orcs implementation.

So it might be preferable - if possible - to abstract away from Hadoop 
implementation. This seems to require some abstraction on Input/Outputstream 
(Seekable/Positionable) and some means to create such a stream. If I understand 
correctly, [this very same 
issue|http://mail-archives.apache.org/mod_mbox/parquet-dev/201802.mbox/%[email protected]%3e]
 was already resolved by Apache Parquet team, so I believe, that should be 
technically possible without breaking changes.

If you also think, this could be doable and possible to be accepted by Apache 
Orc, we could try to implement a PR and actually see whether this approach is 
feasible. 

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

Reply via email to