beam-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Łukasz Gajowy <lgaj...@apache.org>
Subject Re: [DISCUSS] Should File based IOs implement readAll() or just readFiles()
Date Fri, 01 Feb 2019 14:44:47 GMT
+1 to deprecating and not implementing readAll() in transforms where
it is equivalent to matchAll() + readMatches() + readFiles().

It encourages and advertises the use of a nice, composable api reducing the
amount of code to be maintained.

can the Python/Go SDK align with this?

+ 1 to that too.

pt., 1 lut 2019 o 14:56 Ismaël Mejía <iemejia@gmail.com> napisał(a):

> I want to chime in that I am also +1 to deprecating readAll, is there
> anyone strongly pro readAll instead of the explicit composition?
> And more important, can the Python/Go SDK align with this (deprecating
> ReadAll and implementing ReadFiles)?
>
> On Thu, Jan 31, 2019 at 12:34 AM Chamikara Jayalath
> <chamikara@google.com> wrote:
> >
> > Thanks for the clarification Ismaël and Eugene. +1 for deprecating
> existing FooIO.readAll() transforms in favor of FooIO.readFiles().
> >
> > On Wed, Jan 30, 2019 at 3:25 PM Eugene Kirpichov <kirpichov@google.com>
> wrote:
> >>
> >> TextIO.read() and AvroIO.read() indeed perform better than match() +
> readMatches() + readFiles(), due to DWR - so for these two in particular I
> would not recommend such a refactoring.
> >> However, new file-based IOs that do not support DWR should only provide
> readFiles(). Those that do, should provide read() and readFiles(). When SDF
> supports DWR, then readFiles() will be enough in all cases.
> >> In general there's no need for readAll() for new file-based IOs - it is
> always equivalent to matchAll() + readMatches() + readFiles() including
> performance-wise. It was included in TextIO/AvroIO before readFiles() was a
> thing.
> >>
> >> On Wed, Jan 30, 2019 at 2:41 PM Chamikara Jayalath <
> chamikara@google.com> wrote:
> >>>
> >>> On Wed, Jan 30, 2019 at 2:37 PM Chamikara Jayalath <
> chamikara@google.com> wrote:
> >>>>
> >>>>
> >>>>
> >>>> On Wed, Jan 30, 2019 at 2:33 PM Ismaël Mejía <iemejia@gmail.com>
> wrote:
> >>>>>
> >>>>> Ups slight typo, in the first line of the previous email I meant
read
> >>>>> instead of readAll
> >>>>>
> >>>>> On Wed, Jan 30, 2019 at 11:32 PM Ismaël Mejía <iemejia@gmail.com>
> wrote:
> >>>>> >
> >>>>> > Reuven is right for the example, readAll at this moment may
be
> faster
> >>>>> > and also supports Dynamic Work Rebalancing (DWR), but the
> performance
> >>>>> > of the other approach may (and must) be improved to be equal,
once
> the
> >>>>> > internal implementation of TextIO.read moves to a SDF version
> instead
> >>>>> > of the FileBasedSource one, and once that runners support DWR
> through
> >>>>> > SDF. Of course all of this is future work. Probably Eugene
can
> >>>>> > eventually chime in to give more details in practical performance
> in
> >>>>> > his tests in Dataflow.
> >>>>> >
> >>>>> > Really interesting topic, but I want to bring back the discussion
> to
> >>>>> > the subject of the thread. I think there is some confusion
after
> >>>>> > Jeff's example which should have been:
> >>>>> >
> >>>>> >       return input
> >>>>> >           .apply(TextIO.readAll());
> >>>>> >
> >>>>> > to:
> >>>>> >
> >>>>> >       return input
> >>>>> >           .apply(FileIO.match().filepattern(fileSpec))
> >>>>> >           .apply(FileIO.readMatches())
> >>>>> >           .apply(TextIO.readFiles());
> >>>>> >
> >>>>> > This is the question we are addressing, do we need a readAll
> transform
> >>>>> > that replaces the 3 steps or no?
> >>>>
> >>>>
> >>>> Ismaël, I'm not quite sure how these two are equal. readFiles()
> transform returns a PCollection of ReadableFile objects. Users are expected
> to read these files in a subsequent ParDo and produce a PCollection of
> proper type. FooIO.ReadAll() transforms on the other hand are tailored to
> each IO connector and return a PCollection of objects of type that are
> supported to be returned by that IO connector.
> >>>
> >>>
> >>> I assume you meant FileIO.readFiles()  here. Or did you mean
> TextIO.readFiles() ? If so that seems very similar to TextIO.readAll().
> >>>>
> >>>>
> >>>>
> >>>>>
> >>>>> >
> >>>>> > On Wed, Jan 30, 2019 at 9:03 PM Robert Bradshaw <
> robertwb@google.com> wrote:
> >>>>> > >
> >>>>> > > Yes, this is precisely the goal of SDF.
> >>>>> > >
> >>>>> > >
> >>>>> > > On Wed, Jan 30, 2019 at 8:41 PM Kenneth Knowles <klk@google.com>
> wrote:
> >>>>> > > >
> >>>>> > > > So is the latter is intended for splittable DoFn
but not yet
> using it? The promise of SDF is precisely this composability, isn't it?
> >>>>> > > >
> >>>>> > > > Kenn
> >>>>> > > >
> >>>>> > > > On Wed, Jan 30, 2019 at 10:16 AM Jeff Klukas <
> jklukas@mozilla.com> wrote:
> >>>>> > > >>
> >>>>> > > >> Reuven - Is TextIO.read().from() a more complex
case than the
> topic Ismaël is bringing up in this thread? I'm surprised to hear that the
> two examples have different performance characteristics.
> >>>>> > > >>
> >>>>> > > >> Reading through the implementation, I guess the
fundamental
> difference is whether a given configuration expands to TextIO.ReadAll or to
> io.Read. AFAICT, that detail and the subsequent performance impact is not
> documented.
> >>>>> > > >>
> >>>>> > > >> If the above is correct, perhaps it's an argument
for IOs to
> provide higher-level methods in cases where they can optimize performance
> compared to what a user might naively put together.
> >>>>> > > >>
> >>>>> > > >> On Wed, Jan 30, 2019 at 12:35 PM Reuven Lax <relax@google.com>
> wrote:
> >>>>> > > >>>
> >>>>> > > >>> Jeff, what you did here is not simply a refactoring.
These
> two are quite different, and will likely have different performance
> characteristics.
> >>>>> > > >>>
> >>>>> > > >>> The first evaluates the wildcard, and allows
the runner to
> pick appropriate bundling. Bundles might contain multiple files (if they
> are small), and the runner can split the files as appropriate. In the case
> of the Dataflow runner, these bundles can be further split dynamically.
> >>>>> > > >>>
> >>>>> > > >>> The second chops of the files inside the
the PTransform, and
> processes each chunk in a ParDo. TextIO.readFiles currently chops up each
> file into 64mb chunks (hardcoded), and then processes each chunk in a ParDo.
> >>>>> > > >>>
> >>>>> > > >>> Reuven
> >>>>> > > >>>
> >>>>> > > >>>
> >>>>> > > >>> On Wed, Jan 30, 2019 at 9:18 AM Jeff Klukas
<
> jklukas@mozilla.com> wrote:
> >>>>> > > >>>>
> >>>>> > > >>>> I would prefer we move towards option
[2]. I just tried the
> following refactor in my own code from:
> >>>>> > > >>>>
> >>>>> > > >>>>       return input
> >>>>> > > >>>>           .apply(TextIO.read().from(fileSpec));
> >>>>> > > >>>>
> >>>>> > > >>>> to:
> >>>>> > > >>>>
> >>>>> > > >>>>       return input
> >>>>> > > >>>>           .apply(FileIO.match().filepattern(fileSpec))
> >>>>> > > >>>>           .apply(FileIO.readMatches())
> >>>>> > > >>>>           .apply(TextIO.readFiles());
> >>>>> > > >>>>
> >>>>> > > >>>> Yes, the latter is more verbose but not
ridiculously so,
> and it's also more instructive about what's happening.
> >>>>> > > >>>>
> >>>>> > > >>>> When I first started working with Beam,
it took me a while
> to realize that TextIO.read().from() would accept a wildcard. The more
> verbose version involves a method called "filepattern" which makes this
> much more obvious. It also leads me to understand that I could use the same
> FileIO.match() machinery to do other things with filesystems other than
> read file contents.
> >>>>> > > >>>>
> >>>>> > > >>>> On Wed, Jan 30, 2019 at 11:26 AM Ismaël
Mejía <
> iemejia@gmail.com> wrote:
> >>>>> > > >>>>>
> >>>>> > > >>>>> Hello,
> >>>>> > > >>>>>
> >>>>> > > >>>>> A ‘recent’ pattern of use in
Beam is to have in file based
> IOs a
> >>>>> > > >>>>> `readAll()` implementation that basically
matches a
> `PCollection` of
> >>>>> > > >>>>> file patterns and reads them, e.g.
`TextIO`, `AvroIO`.
> `ReadAll` is
> >>>>> > > >>>>> implemented by a expand function
that matches files with
> FileIO and
> >>>>> > > >>>>> then reads them using a format specific
`ReadFiles`
> transform e.g.
> >>>>> > > >>>>> TextIO.ReadFiles, AvroIO.ReadFiles.
So in the end
> `ReadAll` in the
> >>>>> > > >>>>> Java implementation is just an user
friendly API to hide
> FileIO.match
> >>>>> > > >>>>> + ReadFiles.
> >>>>> > > >>>>>
> >>>>> > > >>>>> Most recent IOs do NOT implement
ReadAll to encourage the
> more
> >>>>> > > >>>>> composable approach of File + ReadFiles,
e.g. XmlIO and
> ParquetIO.
> >>>>> > > >>>>>
> >>>>> > > >>>>> Implementing ReadAll as a wrapper
is relatively easy and
> is definitely
> >>>>> > > >>>>> user friendly, but it has an  issue,
it may be error-prone
> and it adds
> >>>>> > > >>>>> more code to maintain (mostly ‘repeated’
code). However
> `readAll` is a
> >>>>> > > >>>>> more abstract pattern that applies
not only to File based
> IOs so it
> >>>>> > > >>>>> makes sense for example in other
transforms that map a
> `Pcollection`
> >>>>> > > >>>>> of read requests and is the basis
for SDF composable style
> APIs like
> >>>>> > > >>>>> the recent `HBaseIO.readAll()`.
> >>>>> > > >>>>>
> >>>>> > > >>>>> So the question is should we:
> >>>>> > > >>>>>
> >>>>> > > >>>>> [1] Implement `readAll` in all file
based IOs to be user
> friendly and
> >>>>> > > >>>>> assume the (minor) maintenance cost
> >>>>> > > >>>>>
> >>>>> > > >>>>> or
> >>>>> > > >>>>>
> >>>>> > > >>>>> [2] Deprecate `readAll` from file
based IOs and encourage
> users to use
> >>>>> > > >>>>> FileIO + `readFiles` (less maintenance
and encourage
> composition).
> >>>>> > > >>>>>
> >>>>> > > >>>>> I just checked quickly in the python
code base but I did
> not find if
> >>>>> > > >>>>> the File match + ReadFiles pattern
applies, but it would
> be nice to
> >>>>> > > >>>>> see what the python guys think on
this too.
> >>>>> > > >>>>>
> >>>>> > > >>>>> This discussion comes from a recent
slack conversation
> with Łukasz
> >>>>> > > >>>>> Gajowy, and we wanted to settle into
one approach to make
> the IO
> >>>>> > > >>>>> signatures consistent, so any opinions/preferences?
>

Mime
View raw message