It would be great I'd it was available for both Java and Python.

On Tue, Jul 2, 2019, 3:57 AM Ismaël Mejía <ieme...@gmail.com> wrote:

> (Adding dev@ and Solomon Duskis to the discussion)
>
> I was not aware of these thanks for sharing David. Definitely it would
> be a great addition if we could have those donated as an extension in
> the Beam side. We can even evolve them in the future to be more FileIO
> like. Any chance this can happen? Maybe Solomon and his team?
>
>
>
> On Tue, Jul 2, 2019 at 9:39 AM David Morávek <d...@apache.org> wrote:
> >
> > Hi, you can use SequenceFileSink and Source, from a BigTable client.
> Those works nice with FileIO.
> >
> >
> https://github.com/googleapis/cloud-bigtable-client/blob/master/bigtable-dataflow-parent/bigtable-beam-import/src/main/java/com/google/cloud/bigtable/beam/sequencefiles/SequenceFileSink.java
> >
> https://github.com/googleapis/cloud-bigtable-client/blob/master/bigtable-dataflow-parent/bigtable-beam-import/src/main/java/com/google/cloud/bigtable/beam/sequencefiles/SequenceFileSource.java
> >
> > It would be really cool to move these into Beam, but that's up to
> Googlers to decide, whether they want to donate this.
> >
> > D.
> >
> > On Tue, Jul 2, 2019 at 2:07 AM Shannon Duncan <
> joseph.dun...@liveramp.com> wrote:
> >>
> >> It's not outside the realm of possibilities. For now I've created an
> intermediary step of a hadoop job that converts from sequence to text file.
> >>
> >> Looking into better options.
> >>
> >> On Mon, Jul 1, 2019, 5:50 PM Chamikara Jayalath <chamik...@google.com>
> wrote:
> >>>
> >>> Java SDK has a HadoopInputFormatIO using which you should be able to
> read Sequence files:
> https://github.com/apache/beam/blob/master/sdks/java/io/hadoop-format/src/main/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIO.java
> >>> I don't think there's a direct alternative for this for Python.
> >>>
> >>> Is it possible to write to a well-known format such as Avro instead of
> a Hadoop specific format which will allow you to read from both
> Dataproc/Hadoop and Beam Python SDK ?
> >>>
> >>> Thanks,
> >>> Cham
> >>>
> >>> On Mon, Jul 1, 2019 at 3:37 PM Shannon Duncan <
> joseph.dun...@liveramp.com> wrote:
> >>>>
> >>>> That's a pretty big hole for a missing source/sink when looking at
> transitioning from Dataproc to Dataflow using GCS as storage buffer instead
> of a traditional hdfs.
> >>>>
> >>>> From what I've been able to tell from source code and documentation,
> Java is able to but not Python?
> >>>>
> >>>> Thanks,
> >>>> Shannon
> >>>>
> >>>> On Mon, Jul 1, 2019 at 5:29 PM Chamikara Jayalath <
> chamik...@google.com> wrote:
> >>>>>
> >>>>> I don't think we have a source/sink for reading Hadoop sequence
> files. Your best bet currently will probably be to use FileSystem
> abstraction to create a file from a ParDo and read directly from there
> using a library that can read sequence files.
> >>>>>
> >>>>> Thanks,
> >>>>> Cham
> >>>>>
> >>>>> On Mon, Jul 1, 2019 at 8:42 AM Shannon Duncan <
> joseph.dun...@liveramp.com> wrote:
> >>>>>>
> >>>>>> I'm wanting to read a Sequence/Map file from Hadoop stored on
> Google Cloud Storage via a " gs://bucket/link/SequenceFile-* " via the
> Python SDK.
> >>>>>>
> >>>>>> I cannot locate any good adapters for this, and the one Hadoop
> Filesystem reader seems to only read from a "hdfs://" url.
> >>>>>>
> >>>>>> I'm wanting to use Dataflow and GCS exclusively to start mixing in
> Beam pipelines with our current Hadoop Pipelines.
> >>>>>>
> >>>>>> Is this a feature that is supported or will be supported in the
> future?
> >>>>>> Does anyone have any good suggestions for this that is performant?
> >>>>>>
> >>>>>> I'd also like to be able to write back out to a SequenceFile if
> possible.
> >>>>>>
> >>>>>> Thanks!
> >>>>>>
>

Reply via email to