Serializable coder had a separate set of issues - often larger and less
efficient. Ideally, we would have an avrocoder.

On Wed, Apr 5, 2017, 2:15 PM Pablo Estrada <pabl...@google.com.invalid>
wrote:

> As a note, it seems that SerializableCoder does the trick in this case, as
> it does not require a no-arg constructor for the class that is being
> deserialized - so perhaps we should encourage people to use that in the
> future.
> Best
> -P.
>
> On Wed, Apr 5, 2017 at 1:48 PM Pablo Estrada <pabl...@google.com> wrote:
>
> > Hi all,
> > I was encouraged to write about my troubles to use PCollections of
> > AutoValue classes with AvroCoder; because it seems like currently, this
> is
> > not possible.
> >
> > As part of the changes to PAssert, I meant to create a SuccessOrFailure
> > class that could be passed in a PCollection to a `concludeTransform`,
> which
> > would be in charge of validating that all the assertions succeeded, and
> use
> > AvroCoder for serialization of that class. Consider this dummy example:
> >
> > @AutoValue
> > abstract class FizzBuzz {
> > ...
> > }
> >
> > class FizzBuzzDoFn extends DoFn<Integer, FizzBuzz> {
> > ...
> > }
> >
> > 1. The first problem was that the abstract class does not have any
> > attributes, so AvroCoder can not scrape them. For this, (with advice from
> > Kenn Knowles), the Coder would need to take the AutoValue-generated
> class:
> >
> > .apply(ParDo.of(new FizzBuzzDoFn()))
> > .setCoder(AvroCoder.of((Class<FizzBuzz>) AutoValue_FizzBuzz.class))
> >
> > 2. This errored out saying that FizzBuzz and AutoValue_FizzBuzz are
> > incompatible classes, so I just tried bypassing the type system like so:
> >
> > .setCoder(AvroCoder.of((Class) AutoValue_FizzBuzz.class))
> >
> > 3. This compiled properly, and encoding worked, but the problem came at
> > decoding, because Avro specifically requires the class to have a no-arg
> > constructor [1], and AutoValue-generated classes do not come with one.
> This
> > is a problem for several serialization frameworks, and we're not the
> first
> > ones to hit this [2], and the AutoValue people don't seem keen on adding
> > this.
> >
> > Considering all that, it seems that the AutoValue-AvroCoder pair can not
> > currently work. We'd need a serialization framework that does not depend
> on
> > calling the no-arg constructor and then filling in the attributes with
> > reflection. I'm trying to check if SerializableCoder has different
> > deserialization techniques; but for PAssert, I just decided to use
> > POJO+AvroCoder.
> >
> > I hope my experience may be useful to others, and maybe start a
> discussion
> > on how to enable users to have AutoValue classes in their PCollections.
> >
> > Best
> > -P.
> >
> > [1] -
> >
> http://avro.apache.org/docs/1.7.7/api/java/org/apache/avro/reflect/package-summary.html?is-external=true
> > [2] - https://github.com/google/auto/issues/122
> >
> >
>

Reply via email to