On Wed, Feb 8, 2017 at 1:27 PM, Eugene Kirpichov <
kirpic...@google.com.invalid> wrote:

> So... Would it be fair to say that everybody would be satisfied if we
> treated the "glorified combine" transforms (Sum, Count, Mean, Sample,
> Latest) the following way:
> - For each case, SDK must expose the relevant CombineFn as a static factory
> function: e.g. Sum.ofIntegers(), Latest.of(), etc. [it may make sense to
> discuss the naming of these CombineFn factory functions so they are not
> confused with neighboring PTransform factory functions]
>

Or just expose the CombineFn class itself, rather than a factory method to
construct it? As for naming (if we go for factories), maybe
Mean.combineFn().


> - Expose also a PTransform factory function, returning either a
> PTransform<InputT, OutputT> (if there's nothing to configure on the
> transform), or its concrete subclass (if there is something to configure).
> Transition from PTransform to subclass is always possible in a
> backward-compatible way, so it's safe to err on the side of returning
> PTransform.
> - Do *not* return concrete types from the PTransform factory function such
> as Combine.Globally - instead, if the user has an advanced use case and
> wants to configure the combine, they should apply Combine.globally()
> themselves to your exposed CombineFn.
>
> In particular, this means:
> - Sum, Mean stay unchanged
> - Count, Sample, Latest should additionally expose their CombineFn's:
> Count.of()? or how should we name them?
> - Count.globally() and Count.perKey() should be changed from returning
> Combine.Globally and Combine.PerKey to returning the more general type
> PTransform<..., ...>. Cases where the user relies on them returning a
> Combine should be changed to applying the Combine manually.
>
> Makes sense?
>

So the value in Mean.perKey() is solely in the fact that it's pithier for
Combine.perKey(Mean.combineFn())? Or do we assume that the former could
possibly get a new implementation, but the latter should be used if
additional configuration is needed?

On Tue, Feb 7, 2017 at 10:50 PM Dan Halperin <dhalp...@google.com.invalid>
> wrote:
>
> > I am generally persuaded to at least change my number to something like 0
> > :). These are pretty reasonable perspectives, especially pointing out
> that
> > withSideInputs is pretty useless in Count ;)
> >
> > On Tue, Feb 7, 2017 at 10:04 PM, Kenneth Knowles <k...@google.com.invalid
> >
> > wrote:
> >
> > >  On Tue, Feb 7, 2017 at 8:43 PM, Eugene Kirpichov <
> > >
> > > > kirpic...@google.com.invalid> wrote:
> > > > I must admit I didn't quite
> > > > understand the option of "implements CombiningTransform".
> > > >
> > >
> > > On Tue, Feb 7, 2017 at 9:04 PM, Robert Bradshaw
> > > <rober...@google.com.invalid
> > > > wrote:
> > >
> > > > Sorry, I'll try to clarify. ... <<codez>>...
> > > >
> > >
> > > FWIW this is also what I meant by my 3b "HasACombineInsideIt". The
> > > difference between my suggestion and Robert's is that I used a self
> type,
> > > which is really not worth the trouble (in fact, it blocked me from
> > bringing
> > > this to fruition last time we had this same conversation).
> > >
> > > Kenn
> > >
> >
>

Reply via email to