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