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().
Pros:
- Style consistency with other transforms. Almost all transforms have their
own class, and their factory functions return that class.
- Implementation can evolve. However, in case of Count, that is 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.

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