On Thu, Oct 13, 2016 at 10:36 PM, Eugene Kirpichov
<kirpic...@google.com.invalid> wrote:
> I think the choice between #1 or #3 is a red herring - the cases where #3
> is a better choice than #1 are few and far between, and probably not at all
> controversial (e.g. ParDo). So I suggest we drop this part of the
> discussion.

I decided to take a peek at the transforms we currently have, and
actually it seems that most of them fall into the category of having
zero or one "primary, required" arguments intrinsic to what the
transform is, and then possibly some optional ones. I'm becoming even
more a fan of #3--it makes it harder for the user to even construct an
invalid transform (and is better documenting too, both on docs and for
IDE autocompletion, about what's essential vs. the slew of optional
things).

We do loose the "database reader ready-to-go" bit, but I'm not sure
that's such a loss. One can instead have

class CompanyDefaults {
    public static DatabaseIO.Read<T> setup(DatabaseIO.Read<T> reader)
}

which is actually superior if DatabaseIO.Read is a base class (or
interface) that may have several implementations.

> Looks like the main contenders for the complex case are #1 (Foo.<T>blah())
> vs. #4 (Foo.Unbound and Foo.Bound).
>
> Dan, can you clarify again what you mean by this:
> "a utility function that gives you a database reader ready-to-go ... but
> independent of the type you want the result to end up as. You can't do
> that if you must bind the type first."
>
> I think this is quite doable with #1:
>
> class CompanyDefaults {
> public static <T> DatabaseIO.Read<T> defaultDatabaseIO() { return
> DatabaseIO.<T>create().withSettings(blah).withCredentials(blah); }
> }
>
> DatabaseIO.Read<String> source =
> CompanyDefaults.<String>defaultDatabaseIO().withQuery(blah);
>
> All in all, it seems to me that #1 (combined with potentially encapsulating
> parts of the configuration into separate objects, such as
> JdbcConnectionParameters in JdbcIO) is the only option that can do
> everything fairly well, its only downside is having to specify the type,

Having to repeat the type is a significant downside, especially when
your types get long. (Yes, I've faced types that get so verbose you
have to figure out where to put the line breaks.) This is why
inference of template arguments was added to the language, and the
whole reason for the existence of many of Guava's "constructors" like
Lists.newArrayList(), etc. (now obsolete due constructors allowing
inference).

> and it is very easy to implement when you're implementing your own
> transform - which, I agree with Kenn, matters a lot too.
>
> I think that coming up with an easy-to-implement, universally applicable
> pattern matters a lot also because the Beam ecosystem is open, and the set
> of connectors/transforms available to users will not always be as carefully
> curated and reviewed as it is currently - the argument "the implementation
> complexity doesn't matter because the user doesn't see it" will not apply.
> So, ultimately, "are there a lot of good-quality connectors available to
> Beam users" will be equivalent to "is it easy to develop a good-quality
> connector". And the consistency between APIs provided by different
> connectors will matter for the user experience, too.

+1

> On Thu, Oct 13, 2016 at 7:09 PM Kenneth Knowles <k...@google.com.invalid>
> wrote:
>
>> On Thu, Oct 13, 2016 at 4:55 PM Dan Halperin <dhalp...@google.com.invalid>
>> wrote:
>> > These
>> > suggestions are motivated by making things easier on transform writers,
>> but
>> > IMO we need to be optimizing for transform users.
>>
>> To be fair to Eugene, he was actually analyzing real code patterns that
>> exists in Beam today, not suggesting new ones.
>>
>> Along those lines, your addition of the BigTableIO pattern is well-taken
>> and my own analysis of that code is #5: "when you don't have a type
>> variable to bind, leave every field blank and validate later. Also, have an
>> XYZOptions object". I believe in the presence of type parameters this
>> reduces to #4 Bound/Unbound classes but it is more palatable in the
>> no-type-variable case. It is also a good choice when varying subsets of
>> parameters might be optional - the Window transform matches this pattern
>> for good reason.
>>
>> The other major drawback of #3 is the inability to provide generic
>> > configuration. For example, a utility function that gives you a database
>> > reader ready-to-go with all the default credentials and options you need
>> --
>> > but independent of the type you want the result to end up as. You can't
>> do
>> > that if you must bind the type first.
>> >
>>
>> This is a compelling use case. It is valuable for configuration to be a
>> first-class object that can be passed around. BigTableOptions is a good
>> example. It isn't in contradiction with #3, but actually fits very nicely.
>>
>> By definition for this default configuration to be first-class it has to be
>> more than an invalid intermediate state of a PTransform's builder methods.
>> Concretely, it would be BigTableIO.defaultOptions(), which would make
>> manifest the inaccessible default options that could be implied by
>> BigTableIO.read(). There can sometimes be a pretty fine line between a
>> builder and an options object, to be sure. You might distinguish it by
>> whether you would conceivably use the object elsewhere - and for
>> BigTableOptions the answer is certainly "yes" since it actually is an
>> external class. In the extreme, every method takes one giant POJO and that
>> sucks.
>>
>>
>> > I think that in general all of these patterns are significantly worse in
>> > the long run than the existing standards, e.g., for BigtableIO.
>>
>>
>> Note that BigTableIO.read() is actually not "ready-to-go" but has nulls and
>> empty strings that cause crashes if they are not overridden. It is just a
>> builder without the concluding "build()" method (for the record: I find
>> concluding "build()" methods pointless, too :-)
>>
>> One of the better examples of the pattern of "ready-to-go" builders -
>> though not a transform - is WindowingStrategy (props to Ben), where there
>> are intelligent defaults and you can override them, and it tracks whether
>> or not each field is a default or a user-specified value. To start it off
>> you have to either request "globalDefault()" or "of(WindowFn)", in the
>> spirit of #3.
>>
>> Kenn
>>
>> On Fri, Oct 7, 2016 at 4:48 PM, Eugene Kirpichov <
>> > kirpic...@google.com.invalid> wrote:
>> >
>> > > In my original email, all FooBuilder's should be simply Foo. Sorry for
>> > the
>> > > confusion.
>> > >
>> > > On Thu, Oct 6, 2016 at 3:08 PM Kenneth Knowles <k...@google.com.invalid
>> >
>> > > wrote:
>> > >
>> > > > Mostly my thoughts are the same as Robert's. Use #3 whenever
>> possible,
>> > > > fallback to #1 otherwise, but please consider using informative names
>> > for
>> > > > your methods in all cases.
>> > > >
>> > > > #1 GBK.<T>create(): IMO this pattern is best only for transforms
>> where
>> > > > withBar is optional or there is no such method, as in GBK. If it is
>> > > > mandatory, it should just be required in the first method, eliding
>> the
>> > > > issue, as in ParDo.of(DoFn<I, O>), MapElements.via(...), etc, like
>> you
>> > > say
>> > > > in your concluding remark.
>> > > >
>> > > > #2 FooBuilder<?> FooBuilder.create(): this too - if you are going to
>> > fix
>> > > > the type, fix it first. If it is optional and Foo<?> is usable as a
>> > > > transform, then sure. (it would have be something weird like
>> > Foo<InputT,
>> > > > OutputT, ?> extends PTransform<InputT, OutputT>)
>> > > >
>> > > > #3 Foo.create(Bar<T>): this is best. Do this whenever possible. From
>> my
>> > > > perspective, instead of "move the param to create(...)" I would
>> > describe
>> > > > this as "delete create() then rename withBar to create". Just skip
>> the
>> > > > second step and you are in an even better design, withBar being the
>> > > > starting point. Just like ParDo.of and MapElements.via.
>> > > >
>> > > > #4 Dislike this, too, for the same reasons as #2 plus code bloat plus
>> > > user
>> > > > confusion.
>> > > >
>> > > > Side note since you use this method in all your examples: This kind
>> of
>> > > use
>> > > > of "create" is a bad method name. There may be no new object
>> "created".
>> > > > Sometimes we have no better idea, but create() is a poor default. For
>> > GBK
>> > > > both are bad: create() (we really only need one instance so why
>> create
>> > > > anything?) and create(<boolean>) (what is the unlabeled boolean?).
>> They
>> > > > would be improved by GBK.standard() and GBK.fewKeys() or some such. I
>> > > tend
>> > > > to think that focusing on this fine polish eliminates a lot of cases
>> > for
>> > > > the generalized question.
>> > > >
>> > > > Kenn
>> > > >
>> > > > On Thu, Oct 6, 2016 at 2:10 PM Eugene Kirpichov
>> > > > <kirpic...@google.com.invalid> wrote:
>> > > >
>> > > > > Quite a few transforms in the SDK are generic (i.e. have type
>> > > > parameters),
>> > > > > e.g. ParDo, GroupByKey, Keys / WithKeys, many connectors (TextIO,
>> > > > KafkaIO,
>> > > > > JdbcIO, MongoDbGridFSIO etc - both read and write). They use
>> > different
>> > > > > styles of binding the type parameters to concrete types in caller
>> > code.
>> > > > >
>> > > > > I would like us to make a decision which of those styles to
>> recommend
>> > > for
>> > > > > new transform and connectors writers. This question is coming up
>> > rather
>> > > > > frequently, e.g. it came up in JdbcIO and MongoDbGridFSIO.
>> > > > >
>> > > > > For the purpose of this discussion, imagine a hypothetical builder
>> > > class
>> > > > > that looks like this:
>> > > > >
>> > > > > class Foo<T> {
>> > > > > private Bar<T> bar;
>> > > > > private int blah;
>> > > > >
>> > > > > Foo<T> withBlah(int blah);
>> > > > > }
>> > > > >
>> > > > > So far I've seen several styles of binding the type argument in a
>> > > > withBar()
>> > > > > method vs. a creation method:
>> > > > >
>> > > > > 1. Binding at the creation method: e.g.:
>> > > > >
>> > > > > class Foo<T> {
>> > > > > ...
>> > > > > public static <T> Foo<T> create();
>> > > > > public FooBuilder<T> withBar(Bar<T> bar);
>> > > > > }
>> > > > >
>> > > > > Foo<String> foo = Foo.<String>create().withBlah(42).withBar(new
>> > > > > StringBar());
>> > > > >
>> > > > > Example: GroupByKey does this. As well as other transforms that
>> don't
>> > > > have
>> > > > > a withBar()-like method, but still need a type argument, e.g. Keys.
>> > > > >
>> > > > > Pros: completely unambiguous, easy to code, interacts well with
>> > > > @AutoValue
>> > > > > Cons: need to specify type once at call site.
>> > > > >
>> > > > > 2. Binding at a method that takes an argument of the given type
>> (let
>> > us
>> > > > > call it "a constraint argument"), e.g.:
>> > > > >
>> > > > > class Foo<T> {
>> > > > > ...
>> > > > > public static FooBuilder<?> create();
>> > > > > public <U> FooBuilder<U> withBar(Bar<U> bar);
>> > > > > }
>> > > > >
>> > > > > Foo<String> foo = Foo.create().withBlah(42).withBar(new
>> StringBar());
>> > > > >
>> > > > > Example: KafkaIO
>> > > > >
>> > > > >
>> > > > https://github.com/apache/incubator-beam/blob/master/sdks/
>> > >
>> >
>> java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L280
>> > > > >
>> > > > > Pros: don't need to specify type at call site.
>> > > > > Cons: doesn't interact well with @AutoValue (it doesn't support
>> > builder
>> > > > > methods that change type) - requires unchecked conversions.
>> > > > >
>> > > > > 3. Forcing to provide a "constraint argument" in the creation
>> method:
>> > > > >
>> > > > > class Foo<T> {
>> > > > > ...
>> > > > > public static <T> FooBuilder<T> create(Bar<T> bar);
>> > > > > // (do not provide withBar)
>> > > > > }
>> > > > >
>> > > > > Foo<String> foo = Foo.create(new StringBar()).withBlah(42);
>> > > > >
>> > > > > Example: WithKeys
>> > > > >
>> > > > >
>> > > > https://github.com/apache/incubator-beam/blob/master/sdks/
>> > > java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
>> > > > > ,
>> > > > > ParDo
>> > > > >
>> > > > > Pros: easy to code, interacts ok with @AutoValue, don't need to
>> > specify
>> > > > > type at call site.
>> > > > > Cons: need to supply all constraint arguments in the create method,
>> > so
>> > > > they
>> > > > > are treated differently from other arguments.
>> > > > >
>> > > > > 4. Splitting the builder into a "bound" and "unbound" class:
>> > > > >
>> > > > > class Foo {
>> > > > > Unbound create();
>> > > > >
>> > > > > class Unbound {
>> > > > > public Unbound withBlah(int blah);
>> > > > > public <T> Bound<T> withBar(Bar<T> bar);
>> > > > > }
>> > > > >
>> > > > > class Bound<T> {
>> > > > > public Bound<T> withBlah(int blah);
>> > > > > }
>> > > > > }
>> > > > >
>> > > > > Foo.Bound<String> foo = Foo.create().withBlah(42).withBar(new
>> > > > StringBar());
>> > > > >
>> > > > > Example: TextIO.Read
>> > > > >
>> > > > >
>> > > > https://github.com/apache/incubator-beam/blob/master/sdks/
>> > > java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
>> > > > > Pros: even more type-safe at call site than the others (using an
>> > > > incomplete
>> > > > > builder is a compile error)
>> > > > > Cons: very cumbersome to implement, lots of confusion between
>> > > "(un)bound"
>> > > > > and "(un)bounded", tempting for clients to use ugly variable names
>> > such
>> > > > as
>> > > > > "Foo.Bound<String> bound = ..." (rather than "foo")
>> > > > >
>> > > > > ****
>> > > > >
>> > > > > I'd like to argue in favor of #1, because:
>> > > > > - It makes sense for transforms like Keys.create() which do not
>> have
>> > a
>> > > > > "constraint argument", so we can have consistency between such
>> > > transforms
>> > > > > and the others.
>> > > > > - It is the simplest to implement, and causes the fewest amount of
>> > > > > generics-related confusion when reading the implementation code.
>> > > > > - It interacts well with @AutoValue builders.
>> > > > >
>> > > > > The only downside is that you have to specify the generic argument
>> at
>> > > > call
>> > > > > site, but I think this is acceptable given the benefits of
>> > consistency,
>> > > > > unambiguity and providing a pattern that's easy to follow for
>> future
>> > > > > transform writers.
>> > > > >
>> > > > > Of course, there should be an exception for cases when there is a
>> > very
>> > > > > small and fixed number of arguments, or when it's clear that the
>> > > > > "constraint argument" is the most important one - e.g.
>> > ParDo.of(DoFn<A,
>> > > > B>)
>> > > > > should *not* be changed to ParDo.<A, B>create().withFn(DoFn<A, B>).
>> > > Also,
>> > > > > I'm not suggesting making changes to existing transforms, only
>> > deciding
>> > > > > which pattern to recommend for new transforms.
>> > > > >
>> > > > > WDYT?
>> > > > >
>> > > >
>> > >
>> >
>>

Reply via email to