+1

On Thu, Dec 8, 2016 at 1:10 PM Jean-Baptiste Onofré <j...@nanthrax.net> wrote:

> +1
>
> Regards
> JB
>
> On 12/07/2016 10:37 PM, Kenneth Knowles wrote:
> > Hi all,
> >
> > I want to bring up another major backwards-incompatible change before it
> is
> > too late, to resolve [BEAM-438].
> >
> > Summary: Leave PInput.apply the same but rename PTransform.apply to
> > PTransform.expand. I have opened [PR #1538] just for reference (it took
> 30
> > seconds using IDE automated refactor)
> >
> > This change affects *PTransform authors* but does *not* affect pipeline
> > authors.
> >
> > This issue was filed a long time ago. It has been a problem many times
> with
> > actual users since before Beam started incubating. This is what goes
> wrong
> > (often):
> >
> >    PCollection<Foo> input = ...
> >    PTransform<PCollection<Foo>, ...> transform = ...
> >
> >    transform.apply(input)
> >
> > This type checks and even looks perfectly normal. Do you see the error?
> >
> > ... what we need the user to write is:
> >
> >     input.apply(transform)
> >
> > What a confusing difference! After all, the first one type-checks and the
> > first one is how you apply a Function or Predicate or
> SerializableFunction,
> > etc. But it is broken. With transform.apply(input) the transform is not
> > registered with the pipeline at all.
> >
> > We obviously can't (and don't want to) change the most core way that
> > pipeline authors use Beam, so PInput.apply (aka PCollection.apply) must
> > remain the same. But we do need a way to make it impossible to mix these
> up.
> >
> > The simplest way I can think of is to choose a new name for the other
> > method involved. Users probably won't write transform.expand(input) since
> > they will never have seen it in any examples, etc. This will just make
> > PTransform authors need to do a global rename, and the type system will
> > direct them to all cases so there is no silent failure possible.
> >
> > What do you think?
> >
> > Kenn
> >
> > [BEAM-438] https://issues.apache.org/jira/browse/BEAM-438
> > [PR #1538] https://github.com/apache/incubator-beam/pull/1538
> >
> > p.s. there is a really amusing and confusing call chain:
> PCollection.apply
> > -> Pipeline.applyTransform -> Pipeline.applyInternal ->
> > PipelineRunner.apply -> PTransform.apply
> >
> > After this change and work to get the runner out of the loop, it becomes
> > PCollection.apply -> Pipeline.applyTransform -> PTransform.expand
> >
>
> --
> Jean-Baptiste Onofré
> jbono...@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com
>

Reply via email to