On Wed, Jun 26, 2019 at 5:46 AM Robert Bradshaw <rober...@google.com> wrote:

> Good question.
>
> I'm not sure what could be done with (5) if it contains no deferred
> objects (e.g there's nothing to wait on).
>
> There is also (6) return PCollection<SourceSpecificWriteResult>. The
> advantage of (2) is that one can migrate to (1) or (6) without
> changing the public API, while giving something to wait on without
> promising anything about its contents.


> I would probably lean towards (4) for anything that would want to
> return multiple signals/outputs (e.g. successful vs. failed writes)
> and view (3) as being a "cheap" but more cumbersome for the user way
> of writing (4). In both cases, more information can be added in a
> forward-compatible way. Technically (4) could extend (3) if one wants
> to migrate from (3) to (4) to provide a nicer API in the future. (As
> an aside, it would be interesting if any of the schema work that lets
> us get rid of tuple tags for elements (e.g. join operations) could let
> us get rid of tuple tags for PCollectionTuples (e.g. letting a POJO
> with PCollection members be as powerful as a PCollectionTuple).
>
> On Wed, Jun 26, 2019 at 2:23 PM Ismaël Mejía <ieme...@gmail.com> wrote:
> >
> > Beam introduced in version 2.4.0 the Wait transform to delay
> > processing of each window in a PCollection until signaled. This opened
> > new interesting patterns for example writing to a database and when
> > ‘fully’ done write to another database.
> >
> > To support this pattern an IO connector Write transform must return a
> > type different from PDone to signal the processing of the next step.
> > Some IOs have already started to implement this return type, but each
> > returned type has different pros and cons so I wanted to open the
> > discussion on this to see if we could somehow find a common pattern to
> > suggest IO authors to follow (Note: It may be the case that there is
> > not a pattern that fits certain use cases).
> >
> > So far the approaches in our code base are:
> >
> > 1. Write returns ‘PCollection<Void>’
> >
> > This is the simplest case but if subsequent transforms require more
> > data that could have been produced during the write it gets ‘lost’.
> > Used by JdbcIO and DynamoDBIO.
> >
> > 2. Write returns ‘PCollection<?>’
> >
> > We can return whatever we want but the return type is uncertain for
> > the user in case he wants to use information from it. This is less
> > user friendly but has the maintenance advantage of not changing
> > signatures if we want to change the return type in the future. Used by
> > RabbitMQIO.
> >
> > 3. Write returns a `PCollectionTuple`
> >
> > It is like (2) but with the advantage of returning an untyped tuple of
> > PCollections so we can return more things. Used by SnsIO.
> >
> > 4. Write returns ‘a class that implements POutput’
> >
> > This class wraps inside of the PCollections that were part of the
> > write, e.g. SpannerWriteResult. This is useful because we can be
> > interested on saving inside a PCollection of failed mutations apart of
> > the ‘done’ signal. Used by BigQueryIO and SpannerIO. A generics case
> > of this one is used by FileIO for Destinations via:
> > ‘WriteFilesResult<DestinationT>’.
> >
> > 5. Write returns ‘a class that implements POutput’ with specific data
> > (no PCollections)
> >
> > This is similar to (4) but with the difference that the returned type
> > contains the specific data that may be needed next, for example not a
> > PCollection but values like the number of rows written. Used by
> > BigtableIO (PR in review at the moment). (This can be seen as a
> > simpler version of 4).
>

Thanks Ismaël for detailing various approaches with examples.

I think current PR for BigTable returns a PCollection<BigTableWriteResult>
from a PTransform 'WithWriteResults' that can be optionally invoked through
a BigTableIO.Write.withWriteResults(). So this is more closer to (6) Robert
mentioned. But (1) was also discussed as an option. PR is
https://github.com/apache/beam/pull/7805 for anybody interested.

I think (6) is less cumbersome to implement/use and allows us to easily
extend the transform through more chaining or by changing the return
transform through additional "with*" methods to the FooIO.Write class.

Thanks,
Cham

>
> > I would like to have your opinions on which approach you think it is
> > better or worse and arguments if you see other
> > advantages/disadvantages. I am probably more in the (4) camp but I
> > feel somehow attracted by the flexibility that the lack of strict
> > typing brings in (2, 3) in case of changes to the public IO API (of
> > course this can be contested too).
> >
> > Any other ideas, preferences, issues we may be missing?
>

Reply via email to