There's 2 points here

1. If Count.globally() is implemented via Combine.globally(), then should
Count.globally() return a Combine.Globally, or should it wrap it into a new
class Count.Globally? (that's what I'm wondering in this thread)

I think the "least visibility" argument here would lead us to saying we
should wrap into a new class, because returning a Combine.Globally leaks
the implementation detail that Count is always implemented via Combine.

2. If Thumbs.twiddle() transforms Foo to Bar, should it return a
PTransform<Foo, Bar>, or should it return Thumbs.Twiddle (which extends
PTransform<Foo, Bar>)? This is where your "least visibility" argument from
that thread applies more - we're exposing a more specific type - but I
think by exposing this type we are not imposing any additional obligations
on ourselves and not leaking any details: saying "twiddle() returns a
Twiddle" is as good as saying nothing.

In case of Combine, the ability to additionally configure it for the user
is a strong argument. But I think we've been through a similar situation
and made a different decision: bounded reads from unbounded sources. We
started off with exposing UnboundedSource because a user might want to
specify .withMaxNumRecords() and .withMaxReadTime() on it, but changed our
minds to saying "if you're developing an unbounded connector, and if you
want to make a promise that this is part of your connector's public API,
then you have to implement these functions yourself". The reasoning was
that it might not stay an UnboundedSource forever - and indeed, with SDF,
it won't, and the implementation of these functions will become custom.

Also, the choice to return a Count.Globally (and propagate the needed
functions from Combine.Globally) makes the assumption that as a developer
of the Count.Globally transform, you fully control its API surface. E.g. by
supporting a .withFanout() function, you're saying "When counting things
globally, you may need to configure the fanout". Corollary: you also
control what functions its API *doesn't* have - e.g. it probably shouldn't
have .withSideInputs() because side inputs make no sense here.

The choice to return a Combine.Globally lets you automatically get all new
APIs that Combine.Globally may ever get - which may or may not be a good
thing. It is good because these functions may be useful; it is bad because
as a developer of Count.Globally it is now your responsibility to
anticipate all possible interactions of parameters users might set on it
with the particular CombineFn you supplied; and it is your responsibility
to keep that up-to-date as Combine.Globally evolves (e.g. it might some day
add a parameter that interacts badly with CountFn and then you probably
should warn users not to set it). Granted, in this particular case it's
easy, but we're talking about the general principle.

On Tue, Feb 7, 2017 at 5:45 PM Dan Halperin <dhalp...@google.com.invalid>
wrote:

> A little bit more inline:
>
> On Tue, Feb 7, 2017 at 5:15 PM, Eugene Kirpichov <
> kirpic...@google.com.invalid> wrote:
>
> > Hello,
> >
> > I was auditing Beam for violations of PTransform style guide
> > https://beam.apache.org/contribute/ptransform-style-guide/ and came
> across
> > another style point that deserves discussion.
> >
> > Look at Count transform:
> >
> >   public static <T> Combine.Globally<T, Long> globally() {
> >     return Combine.globally(new CountFn<T>());
> >   }
> >
> >   public static <K, V> Combine.PerKey<K, V, Long> perKey() {
> >     return Combine.<K, V, Long>perKey(new CountFn<V>());
> >   }
> >
> >   public static <T> PerElement<T> perElement() {
> >     return new PerElement<>();
> >   }
> >
> > I asked myself: should globally() and perKey() also define wrapper
> classes
> > - e.g. should it be "public static <T> Globally<T, Long> globally()"
> where
> > "Globally" is a new inner class of Count?
> >
> > I argue that the answer is yes, but it's not clear-cut.
> > Cons:
> > - If we return a Combine.Globally, the user can use the features provided
> > by Combine.Globally - e.g. .withDefaults(), .withFanout(),
> > .asSingletonView().
> >
>
> +1 to this point, which was a conscious decision in the pre-Beam days
> (which of course means it IS worth revisiting ;).
> Trying to replay the reasoning:
>
> * If wrapping, the author of a new Count.Globally can now only make the
> extra functionality in Combine available by similarly exposing all such
> functions.
>
> * Conversely, the current implementation makes new functionality in Combine
> available "for free" to users of Count.globally(). Whereas new
> functionality on Combine would otherwise mandate that *all wrappers* change
> to actually be exposed.
>
> * There's almost no data here, but: we have added new functionality to
> Combine (withSideInputs) and have not added new functionality to Count.
>
>
> > Pros:
> > - Style consistency with other transforms. Almost all transforms have
> their
> > own class, and their factory functions return that class.
> >
>
> IMO this should only happen if the user needs that class. For all examples
> I'm aware of,
> the returned class has stuff you need to do, like configuring coders or
> side inputs or other parameters.
> IMO if the user need not configure, return the least constraining thing you
> can.
>
>
> > - Implementation can evolve. However, in case of Count, that is unlikely.
> >
>
> +1 to "unlikely"
>
>
> > - ...Or is it? If the transform has a concrete class, then the runner can
> > intercept that class and e.g. provide an (even) more efficient
> > implementation of Count.Globally. This gets much more awkward if the
> runner
> > has to intercept every Combine and check whether it's combining using a
> > CountFn.
> >
>
> Post-runner API, this argument goes away. Instead, the runner will need to
> inspect generically the attributes of the transform to do the replacement.
>
> Summarizing: I currently am a -0.8 on this proposal.
>
>
> >
> > So, I propose to add this as a style guide rule, and solve the problem in
> > "Cons" by saying "Yeah, if you want the extra features of the transform
> > you're expanding into, you have to propagate them through the API of your
> > transform and delegate to the underlying transform manually".
> >
> > Thoughts?
> >
>

Reply via email to