A "fluent" API isn't completely incompatible with our current apply-based
API. We could easily add fluent member functions to PCollections which are
syntactic sugar (i.e. delegate to apply). We would need to be disciplined
though, as there will be a tendency for everyone to ask for their transform
to be added as well (this would be a lot saner in a language that supported
mixin methods). This does have some advantages in cleaner user code and
more discoverable transform (i.e. IDE autocomplete and dropdowns work).

One potential concern would be losing some type safety. e.g. today if I
have a PCollection<Long>, I can't apply GroupByKey to it - the Java type
system will only allow me to do this if I have a Pollection<KV>. If however
groupByKey was a method on PCollection, then we can't stop it from being
called.


On Sun, Mar 11, 2018 at 1:18 PM Jean-Baptiste Onofré <j...@nanthrax.net>
wrote:

> Hi Romain,
>
> I remember we have discussed about the way to express pipeline while ago.
>
> I was fan of a "DSL" compared to the one we have in Camel: instead of
> using apply(), use a dedicated form (like .map(), .reduce(), etc, AFAIR,
> it's the approach in flume).
> However, we agreed that apply() syntax gives a more flexible approach.
>
> Using Java Stream is interesting but I'm afraid we would have the same
> issue as the one we identified discussing "fluent Java SDK". However, we
> can have a Stream API DSL on top of the SDK IMHO.
>
> Regards
> JB
>
> On 11/03/2018 19:46, Romain Manni-Bucau wrote:
> > Hi guys,
> >
> > don't know if you already experienced using java Stream API as a
> > replacement for pipeline API but did some tests:
> > https://github.com/rmannibucau/jbeam
> >
> > It is far to be complete but already shows where it fails (beam doesn't
> > have a way to reduce in the caller machine for instance, coder handling
> > is not that trivial, lambda are not working well with default Stream API
> > etc...).
> >
> > However it is interesting to see that having such an API is pretty
> > natural compare to the pipeline API
> > so wonder if beam should work on its own Stream API (with surely another
> > name for obvious reasons ;)).
> >
> > Romain Manni-Bucau
> > @rmannibucau <https://twitter.com/rmannibucau> | Blog
> > <https://rmannibucau.metawerx.net/> | Old Blog
> > <http://rmannibucau.wordpress.com> | Github
> > <https://github.com/rmannibucau> | LinkedIn
> > <https://www.linkedin.com/in/rmannibucau> | Book
> > <
> https://www.packtpub.com/application-development/java-ee-8-high-performance
> >
>

Reply via email to