Hello,

I've worked on a few different things recently and ran repeatedly into the
same issue: that we do not have clear guidance on who should set the Coder
on a PCollection: is it responsibility of the PTransform that outputs it,
or is it responsibility of the user, or is it sometimes one and sometimes
the other?

I believe that the answer is "it's responsibility of the transform" and
moreover that  ideally PCollection.setCoder() should not exist. Instead:

- Require that all transforms set a Coder on the PCollection's they produce
- i.e. it should never be responsibility of the user to "fix up" a coder on
a PCollection produced by a transform.

- Since all transforms are composed of primitive transforms, saying
"transforms must set a Coder" means simply that all *primitive* transforms
must set a Coder on their output.

- In some cases, a primitive PTransform currently doesn't have enough
information to infer a coder for its output collection - e.g.
ParDo.of(DoFn<InputT, OutputT>) might be unable to infer a coder for
OutputT. In that case such transforms should allow the user to provide a
coder: ParDo.of(DoFn).withOutputCoder(...) [note that this differs from
requiring the user to set a coder on the resulting collection]

- Corollary: composite transforms need to only configure their primitive
transforms (and composite sub-transforms) properly, and give them a Coder
if needed.

- Corollary: a PTransform with type parameters <FooT, BarT, ...> needs to
be configurable with coders for all of these, because the implementation of
the transform may change and it may introduce intermediate collections
involving these types. However, in many cases, some of these type
parameters appear in the type of the transform's input, e.g. a
PTransform<PCollection<KV<FooT, BarT>>, PCollection<MooT>> will always be
able to extract the coders for FooT and BarT from the input PCollection, so
the user does not need to provide them. However, a coder for BarT must be
provided. I think in most cases the transform needs to be configurable only
with coders for its output.

Here's a smooth migration path to accomplish the above:
- Make PCollection.createPrimitiveOutputInternal() take a Coder.
- Make all primitive transforms optionally configurable with a coder for
their outputs, such as ParDo.of(DoFn).withOutputCoder().
- By using the above, make all composite transforms shipped with the SDK
set a Coder on the collections they produce; in some cases, this will
require adding a withSomethingCoder() option to the transform and
propagating that coder to its sub-transforms. If the option is unset,
that's fine for now.
- As a result of the above, get rid of all setCoder() calls in the Beam
repo. The call will still be there, but it will just not be used anywhere
in the SDK or examples, and we can mark it deprecated.
- Add guidance to PTransform Style Guide in line with the above

Does this sound like a good idea? I'm not sure how urgent it would be to
actually do this, but I'd like to know whether people agree that this is a
good goal in general.

Reply via email to