Something similar was discussed a while ago, and lead to the suggestion in
PTransform Style Guide:
https://beam.apache.org/contribute/ptransform-style-guide/#setting-coders-on-output-collections

This suggestion is currently not followed by ParDo, but your plan moves in
that direction, so +1 to that.
Remembering that ParDo's may have multiple outputs, though, I'd suggest to
organize it using builder methods:
ParDo.of(new MyFn())
  .withOutputTags(...)
  .withCoder(...coder for main output...)
  .withCoder(tag1, coder1)
  .withCoder(tag2, coder2)

This would bring ParDo to be similar to all other transforms that allow
specifying a coder.

On Tue, Feb 20, 2018 at 3:41 AM Jean-Baptiste Onofré <[email protected]>
wrote:

> Got the point.
>
> No problem for me. Could be a new core PTransform in core or extension.
>
> Regards
> JB
> Le 20 févr. 2018, à 11:17, Romain Manni-Bucau <[email protected]> a
> écrit:
>>
>> Yep, idea is to encapsulate the transfo. Today if you dev a dofn you are
>> enforced to do a ptransform to force the coder which is a bit overkill in
>> general. Being able to do it on the pardo would increase the composability
>> on the user side and reduce the boilerplate needed for that.
>>
>> public class MyFn extends DoFn<A, B> {
>>
>>   // ...impl
>>
>>  public static PTransform<PCollection<A>, PCollection<B>> of(*...*) {
>>       return ParDo.of(MyCoder.of(), new MyFn());
>>   }
>>
>> }
>>
>> Instead of having to also impl in all fn library:
>>
>> @NoArgsConstructor(access = PROTECTED)
>> @AllArgsConstructor
>> class ParDoTransformCoderProvider<A, B> extends PTransform<PCollection<A>, 
>> PCollection<B>> {
>>
>>     private Coder<B> coder;
>>
>>     private DoFn<A, B> fn;
>>
>>     @Override
>>     public PCollection<B> expand(final PCollection<A> input) {
>>         return input.apply(ParDo.of(fn)).setCoder(coder);
>>     }
>> }
>>
>>
>>
>>
>> Romain Manni-Bucau
>> @rmannibucau <https://twitter.com/rmannibucau> |   Blog
>> <https://rmannibucau.metawerx.net/> | Old Blog
>> <http://rmannibucau.wordpress.com> |  Github
>> <https://github.com/rmannibucau> | LinkedIn
>> <https://www.linkedin.com/in/rmannibucau> | Book
>> <https://www.packtpub.com/application-development/java-ee-8-high-performance>
>>
>> 2018-02-20 11:03 GMT+01:00 Jean-Baptiste Onofré <[email protected]>:
>>
>>> Not on the PCollection ? Only ParDo ?
>>> Le 20 févr. 2018, à 10:50, Romain Manni-Bucau < [email protected]>
>>> a écrit:
>>>>
>>>> Hi guys,
>>>>
>>>> any objection to allow to pass with the pardo a coder? Idea is to avoid
>>>> to have to write your own transform to be able to configure the coder when
>>>> you start from a dofn and just do something like
>>>>
>>>> ParDo.of(new MyFn(), new MyCoder()) which is directly integrable into a
>>>> pipeline properly.
>>>>
>>>> wdyt?
>>>>
>>>> Romain Manni-Bucau
>>>> @rmannibucau <https://twitter.com/rmannibucau> |   Blog
>>>> <https://rmannibucau.metawerx.net/> | Old Blog
>>>> <http://rmannibucau.wordpress.com> |  Github
>>>> <https://github.com/rmannibucau> | LinkedIn
>>>> <https://www.linkedin.com/in/rmannibucau> | Book
>>>> <https://www.packtpub.com/application-development/java-ee-8-high-performance>
>>>>
>>>
>>

Reply via email to