I like the idea of "if you're really just a Combine, then expose (only) the
CombineFn".

However, in case of Count, another argument is that Count.perElement()
already returns a PerElement transform, and it'd be awkward if globally()
and perKey() were only exposed as CombineFn's. In a sense, if you are or
might be developing a family of transforms, creating transform wrapper
classes is a common denominator.

So we have 3 options for Count.globally() and .perKey():
1. Return Combine.Globally, Combine.PerKey transforms (status quo)
2. Create wrapper classes Count.Globally, Count.PerKey
3. Expose only the respective CombineFn and have users apply it themselves

It sounds like I am in favor of a mix of 2 and 3; Dan (so far) is in favor
of 1; Kenn and Ben and you in favor of 3? I must admit I didn't quite
understand the option of "implements CombiningTransform".

On Tue, Feb 7, 2017 at 8:38 PM Robert Bradshaw <rober...@google.com.invalid>
wrote:

> On Tue, Feb 7, 2017 at 7:49 PM, Kenneth Knowles <k...@google.com.invalid>
> wrote:
>
> > I am +0.7 on this idea. My rationale is contained in this thread, but I
> > thought I would paraphrase it anyhow:
> >
> > "You automatically get all the features of Combine" / "If you add a
> feature
> > to Combine you have to update all wrappers"
> >
> > 0. I have been in some of the mentioned historical discussions and was
> > swayed enough to not bother trying to change things, but I don't actually
> > think the abstraction breakage is worth the benefits. I think the status
> > quo is hanging on because it is the status quo.
> >
> > 1. Any Combine wrapper should also be providing its CombineFn, for use
> with
> > state. So the user already has a good way to get the same result without
> > the wrapper, e.g. Combine.globally(Mean.of()). There are reasons to
> provide
> > a factory method, such as abstracting away tweaking of defaults, but I
> > don't know of any cases of this happening.
> >
>
> +1, we need to also be exposing pure CombineFns.
>
>
> > 2. Most interesting transforms will not just be Combine wrappers - we
> have
> > a lot of them because we try to write pithy examples so I bet this issue
> is
> > a bigger deal to us than to our users. I don't have data to back this up;
> > maybe users do write a bunch of Combine wrappers, but IMO they should
> spend
> > their time elsewhere. I certainly would. So the burden of people updating
> > their Combine wrappers is not compelling to me.
> >
>
> Agree here too, I don't think it's worth encouraging creating an extra
> suite of classes just to avoid the Combine.perKey(...) and
> Combine.globally(...). Also, as options are added (like hot key fanout)
> it's likely plumbing these options through will be inconsistently done (as
> well as a lot of effort).
>
>
> > 2a. Consider also Count.perElement(). It really is a glorified combine,
> but
> > none of the supposed benefits of automatic propagation of tuning knobs
> can
> > by accrue to it, because it is not _just_ a Combine.
> >
> > 3. Those transforms that really are just Combine wrappers will not
> require
> > update when there is a new feature added to Combine. A new performance
> > tuning knob, perhaps, but as Eugene points out, new features like side
> > inputs or access to pipeline options aren't automatically applicable even
> > if your transform is a glorified CombineFn.
> >
> > 3a. And since the argument is only about universally applicable builder
> > methods on Combine, how many more do you imagine we will be adding?
> >
> > 3b. You can use type hackery to ensure you don't forget a knob, along the
> > lines of `CountPerElement implements HasACombineInsideIt<
> > CountPerElement>`.
> > This would actually be a step towards enabling e.g. hot key counting in
> > Count.PerElement
> >
>
> Yes, more below.
>
>
> > 4. If a transform that is just a wrapper needs to evolve into more, then
> > you have to make a backwards incompatible change because you broke your
> > abstractions, and you incur all the manual programming cost anyhow.
> >
>
> I was thinking that the Java-esque way of doing things is to provide an
> interface, CombiningTransform<PIC, PCO>, which the CombinePerKey (and
> CombineGlobally) implement. CombiningTransform would have methods like
> withFanout that return a CombiningTransform of the same type. PTransforms
> like PerElement and RemoveDuplicates could implement this as well, despite
> not being subclasses of the Combine ones, and delegate to their inner
> Combine(s) as makes sense. Factory methods like Sum.integersGlobally()
> would be declared to return a CombiningTransform of the right type, and the
> default implementation would be to return a CombineGlobally but there is no
> lock in (other than to support the CombiningTransform methods).
>
> If I were to write a style guide, I might summarize as:
> >
> > (a) If you are writing an interesting CombineFn, give it some public
> > factory methods.
> >
> > (b) If you are writing a transform that _by definition, forever and
> always_
> > is a wrapper on Combine, stop after (a)
> >
> > (c) If you are writing a transform that has conceivably multiple
> > implementation choices, you need an abstraction boundary to protect
> against
> > the current decision.
> >
> > (d) Only return a class more specific than PTransform if your transform
> has
> > more-specific methods to call.
> >
> >
> > All that said, I'm not sure how much influence this particular collection
> > of guidelines will have, hence I don't feel a full +1. (except (a) which
> is
> > very important).
> >
>
> Your summary sounds more like a -1 to the proposal.
>
>
> > On Tue, Feb 7, 2017 at 5:59 PM, Eugene Kirpichov <
> > kirpic...@google.com.invalid> wrote:
> >
> > > 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