Hi Jesse,

+1 to Dan - I think it makes sense to return the specific type
corresponding to the given transform (e.g. returning Combine.Globally from
Combine.globally()), because it very often serves as a builder for adding
more parameters.

You mentioned users extending transform classes. I believe users should not
be doing that (PTransform classes should be final whenever possible), and
inheritance should never be used as an extensibility device for PTransforms
- instead, PTransform composition should.

P.S. I have a "PTransform style guide" in the works that I'll send out for
public review early next week; it mentions this and other points, and
perhaps class member placement will also make sense to add there.

On Fri, Jan 27, 2017 at 9:01 AM Dan Halperin <[email protected]>
wrote:

> On Fri, Jan 27, 2017 at 8:41 AM, Jesse Anderson <[email protected]>
> wrote:
>
> > @dan I thought you were talking about the transform class definition:
> >   public static class GroupedValues<K, InputT, OutputT>
> >       extends PTransform
> >                         <PCollection<? extends KV<K, ? extends
> > Iterable<InputT>>>,
> >                          PCollection<KV<K, OutputT>>> {
> >
>
> If the user needs to call functions on the returned type (in this case,
> Combine.groupedValues() returns a GroupedValues, which allows the user to
> configure side inputs using GroupedValues#withSideInputs), then both:
>
> * the function groupedValues() needs to return a GroupedValues, so that the
> calling code can access methods like GroupedValues#withSideInputs.
> * the class GroupedValues needs to be public, so that the above works.
>
> and, also, as a matter of practice,
>
> * Comprehensive Javadoc should be class-level on public transforms,
> especially when there's many factory methods for these transforms.
>
> Dan
>
>
> >
> >
> > On Fri, Jan 27, 2017 at 11:30 AM Dan Halperin
> <[email protected]
> > >
> > wrote:
> >
> > > Hi Jesse, can you specifically say which functions on Combine and Count
> > > you're thinking of? I believe these transforms are consistent with the
> > > "principle of least visibility" -- make nothing more public than it
> needs
> > > to be.
> > >
> > > Look at Combine.globally
> > > <
> > > https://github.com/apache/beam/blob/master/sdks/java/
> > core/src/main/java/org/apache/beam/sdk/transforms/Combine.java#L124
> > > >.
> > > It returns a Globally, but that is because Globally has a useful public
> > API
> > > surface, adding functions like asSingletonView. I believe similar
> > reasoning
> > > applies to Count.
> > >
> > > However, for cases where the user will not further configure the return
> > > value, it makes sense to return the most public type we can.
> > >
> > > On Fri, Jan 27, 2017 at 6:39 AM, Jesse Anderson <[email protected]
> >
> > > wrote:
> > >
> > > > One con to making transform classes be private would be that it is a
> > > > breaking change. If anyone uses that class directly or extends that
> > > class,
> > > > we'd be breaking that.
> > > >
> > > > On Fri, Jan 27, 2017 at 9:37 AM Jesse Anderson <
> [email protected]>
> > > > wrote:
> > > >
> > > > > Continuing a discussion <https://github.com/apache/beam/pull/1830>
> > > Dan,
> > > > > Kenn, and I were having here since the bug is closed. They pointed
> > out
> > > > > three things:
> > > > >
> > > > >    - Where the private constructor gets placed in the class
> > > > >    - Where the code samples of how to use the class get placed (in
> > the
> > > > >    Transform versus in the static method)
> > > > >    - Are transform classes public or private
> > > > >
> > > > > I noted that those were inconsistent in the code. When I write a
> new
> > > > > transform, I use one of the already written transforms as the
> basis.
> > > > >
> > > > > Looking at Combine and Count:
> > > > >
> > > > >    - The private constructor is at the top of the class
> > > > >    - The code sample is in the Transform class
> > > > >    - The transform class is marked as public
> > > > >
> > > > > I don't have a strong opinion on private constructor and transform
> > > being
> > > > > marked as private/public. I think we should standardize on placing
> > code
> > > > > samples in the static helper methods. That's where people are
> looking
> > > > when
> > > > > using these methods.
> > > > >
> > > > > I think we need to do a general pass to make these consistent after
> > we
> > > > > decide on how they should be done.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jesse
> > > > >
> > > >
> > >
> >
>

Reply via email to