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é <j...@nanthrax.net>:

> Not on the PCollection ? Only ParDo ?
> Le 20 févr. 2018, à 10:50, Romain Manni-Bucau <rmannibu...@gmail.com> 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