[
https://issues.apache.org/jira/browse/PARQUET-1142?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16308295#comment-16308295
]
Werner Daehn edited comment on PARQUET-1142 at 1/2/18 4:24 PM:
---------------------------------------------------------------
Thanks Ryan, I believe this is one of the most important areas where Parquet
can get better.
Do you happen to have some sample code on how to use that? My current approach
is to extend ParquetWriter<T> including the inner Builder class extending
ParquetWriter.Builder<T, Builder<T>>.
Where I am stuck is the Builder.WriteSupport with its Hadoop Configuration
parameter. It has to be implemented but you talk about ParquetReadOptions?
And my class needs a constructor but all visible super()'s are Hadoop based.
Any hints would be greatly appreciated.
Background:
In my world there is no such thing as Hadoop but many different kinds of file
storage options, HDFS being one of those. Therefore I would be in favor of
removing all Hadoop related stuff from Parquet and rather have a
java.io.OutputStream (=org.apache.parquet.io.OutputFile) which then can be used
to store files in different locations, HDFS full client, HDFS web client, S3,
local file system.
My ultimate goal for the moment is to have a AvroParquetWriter without Hadoop
to save to a local file system in a streaming way. This would then be used to
consume the Kafka data of the last 24 hours and write one large Parquet file.
Next step would be a matching reader that does pushdown simple filter criteria.
Best would be one that allows even primary key lookups in a reasonable amount
of time.
was (Author: wdaehn):
Thanks Ryan, I believe this is one of the most important areas where Parquet
can get better.
Do you happen to have some sample code on how to use that? My current approach
is to extend ParquetWriter<T> including the an inner Builder class extending
ParquetWriter.Builder<T, Builder<T>>.
Where I am stuck is the Builder.WriteSupport with its Hadoop Configuration
parameter. It has to be implemented but you talk about ParquetReadOptions?
And my class needs a constructor but all visible super()'s are Hadoop based.
Any hints would be greatly appreciated.
Background:
In my world there is no such thing as Hadoop but many different kinds of file
storage options, HDFS being one of those. Therefore I would be in favor of
removing all Hadoop related stuff from Parquet and rather have a
java.io.OutputStream (=org.apache.parquet.io.OutputFile) which then can be used
to store files in different locations, HDFS full client, HDFS web client, S3,
local file system.
My ultimate goal for the moment is to have a AvroParquetWriter without Hadoop
to save to a local file system in a streaming way. This would then be used to
consume the Kafka data of the last 24 hours and write one large Parquet file.
Next step would be a matching reader that does pushdown simple filter criteria.
Best would be one that allows even primary key lookups in a reasonable amount
of time.
> Avoid leaking Hadoop API to downstream libraries
> ------------------------------------------------
>
> Key: PARQUET-1142
> URL: https://issues.apache.org/jira/browse/PARQUET-1142
> Project: Parquet
> Issue Type: Improvement
> Components: parquet-mr
> Affects Versions: 1.9.0
> Reporter: Ryan Blue
> Assignee: Ryan Blue
> Fix For: 1.10.0
>
>
> Parquet currently leaks the Hadoop API by requiring callers to pass {{Path}}
> and {{Configuration}} instances, and by using Hadoop codecs. {{InputFile}}
> and {{SeekableInputStream}} add alternatives to Hadoop classes in some parts
> of the read path, but this needs to be extended to the write path and to
> avoid passing options through {{Configuration}}.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)