You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@beam.apache.org by Cristian Constantinescu <ze...@gmail.com> on 2021/10/28 18:10:20 UTC

Potential bug: AutoValue + Memoized fields

Hi everyone,

Looks like Beam has a little bit of an issue when using AutoValues with
Memoized (cached) fields. It's not a big issue, and the workaround is
simply not using Memoised fields at the cost of a little performance. (See
comment in code snippet)

The code further below produces this exception:
Exception in thread "main" java.lang.NullPointerException
at
org.apache.beam.sdk.schemas.utils.JavaBeanUtils.createGetter(JavaBeanUtils.java:155)
at
org.apache.beam.sdk.schemas.utils.JavaBeanUtils.lambda$getGetters$1(JavaBeanUtils.java:145)
at
java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
at
java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655)
at
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)
at
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
at
java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
at
java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
at
java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
at
org.apache.beam.sdk.schemas.utils.JavaBeanUtils.lambda$getGetters$2(JavaBeanUtils.java:146)
at
java.base/java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1705)
at
org.apache.beam.sdk.schemas.utils.JavaBeanUtils.getGetters(JavaBeanUtils.java:140)
at
org.apache.beam.sdk.schemas.AutoValueSchema.fieldValueGetters(AutoValueSchema.java:72)
at org.apache.beam.sdk.schemas.CachingFactory.create(CachingFactory.java:56)
at org.apache.beam.sdk.values.RowWithGetters.<init>(RowWithGetters.java:66)
at
org.apache.beam.sdk.values.Row$Builder.withFieldValueGetters(Row.java:835)
at
org.apache.beam.sdk.schemas.GetterBasedSchemaProvider$ToRowWithValueGetters.apply(GetterBasedSchemaProvider.java:64)
at
org.apache.beam.sdk.schemas.GetterBasedSchemaProvider$ToRowWithValueGetters.apply(GetterBasedSchemaProvider.java:49)
at org.apache.beam.sdk.schemas.SchemaCoder.encode(SchemaCoder.java:124)
at org.apache.beam.sdk.coders.Coder.encode(Coder.java:136)
at
org.apache.beam.sdk.util.CoderUtils.encodeToSafeStream(CoderUtils.java:85)
at org.apache.beam.sdk.util.CoderUtils.encodeToByteArray(CoderUtils.java:69)
at org.apache.beam.sdk.util.CoderUtils.encodeToByteArray(CoderUtils.java:54)
at
org.apache.beam.sdk.transforms.Create$Values$CreateSource.fromIterable(Create.java:413)
at org.apache.beam.sdk.transforms.Create$Values.expand(Create.java:370)
at org.apache.beam.sdk.transforms.Create$Values.expand(Create.java:277)
at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:548)
at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:482)
at org.apache.beam.sdk.values.PBegin.apply(PBegin.java:44)
at org.apache.beam.sdk.Pipeline.apply(Pipeline.java:177)
at org.whatever.testing.App.main(App.java:24)


package org.whatever.testing;

import com.google.auto.value.AutoValue;
import com.google.auto.value.extension.memoized.Memoized;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.sdk.schemas.annotations.SchemaCreate;
import org.apache.beam.sdk.schemas.transforms.Convert;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.values.TypeDescriptor;

import java.util.Arrays;

public class App {

    public static void main(String[] args) {
        var options =
PipelineOptionsFactory.fromArgs(args).withValidation().create();

        var p = Pipeline.create(options);
        p

.apply(Create.of(Arrays.asList(FooAutoValue.builder().setDummyProp("dummy").build())))
                .apply(Convert.to(FooAutoValue.class))

.apply(MapElements.into(TypeDescriptor.of(FooAutoValue.class)).via(i
-> {
                    System.out.println(i.toString());
                    return i;
                }))
        ;
        p.run().waitUntilFinish();
    }

    @AutoValue
    @DefaultSchema(AutoValueSchema.class)
    public static abstract class FooAutoValue {
        public abstract String getDummyProp();

        @Memoized // <-- commenting this line makes everything work
        public String getSomething(){
            return "sldj";
        }

        @SchemaCreate
        public static FooAutoValue create(String dummyProp) {
            return builder()
                    .setDummyProp(dummyProp)
                    .build();
        }

        public static Builder builder() {
            return new AutoValue_App_FooAutoValue.Builder();
        }

        @AutoValue.Builder
        public abstract static class Builder {
            public abstract Builder setDummyProp(String newDummyProp);

            public abstract FooAutoValue build();
        }
    }
}


From what I can see, instead of getting the fields from the abstract class,
it's trying to get the fields from the concrete implementation. Given that
they are not abstract (as it's custom for AutoValue classes, no fields are
fetched). This code is in AutoValueSchema.get method.

If confirmed, please create a jira, and I'll gladly look into it, along
with all the other potential bugs I have sent out recently.

Thank you,
Cristian

Re: Potential bug: AutoValue + Memoized fields

Posted by Luke Cwik <lc...@google.com>.
Thanks for the bug report, I filed
https://issues.apache.org/jira/browse/BEAM-13151

You might be interested in https://issues.apache.org/jira/browse/BEAM-10509
as well.

I couldn't find an obvious JIRA user id based upon the name, do you have a
JIRA user id already?
If not, please create one and share it and I can add you as a contributor
to the BEAM project allowing you to assign issues to yourself. Note that
you can always create issues for Apache projects, all you need is the JIRA
account which anyone can sign up for.



On Thu, Oct 28, 2021 at 11:10 AM Cristian Constantinescu <ze...@gmail.com>
wrote:

> Hi everyone,
>
> Looks like Beam has a little bit of an issue when using AutoValues with
> Memoized (cached) fields. It's not a big issue, and the workaround is
> simply not using Memoised fields at the cost of a little performance. (See
> comment in code snippet)
>
> The code further below produces this exception:
> Exception in thread "main" java.lang.NullPointerException
> at
> org.apache.beam.sdk.schemas.utils.JavaBeanUtils.createGetter(JavaBeanUtils.java:155)
> at
> org.apache.beam.sdk.schemas.utils.JavaBeanUtils.lambda$getGetters$1(JavaBeanUtils.java:145)
> at
> java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
> at
> java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655)
> at
> java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)
> at
> java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
> at
> java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
> at
> java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
> at
> java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
> at
> org.apache.beam.sdk.schemas.utils.JavaBeanUtils.lambda$getGetters$2(JavaBeanUtils.java:146)
> at
> java.base/java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1705)
> at
> org.apache.beam.sdk.schemas.utils.JavaBeanUtils.getGetters(JavaBeanUtils.java:140)
> at
> org.apache.beam.sdk.schemas.AutoValueSchema.fieldValueGetters(AutoValueSchema.java:72)
> at
> org.apache.beam.sdk.schemas.CachingFactory.create(CachingFactory.java:56)
> at org.apache.beam.sdk.values.RowWithGetters.<init>(RowWithGetters.java:66)
> at
> org.apache.beam.sdk.values.Row$Builder.withFieldValueGetters(Row.java:835)
> at
> org.apache.beam.sdk.schemas.GetterBasedSchemaProvider$ToRowWithValueGetters.apply(GetterBasedSchemaProvider.java:64)
> at
> org.apache.beam.sdk.schemas.GetterBasedSchemaProvider$ToRowWithValueGetters.apply(GetterBasedSchemaProvider.java:49)
> at org.apache.beam.sdk.schemas.SchemaCoder.encode(SchemaCoder.java:124)
> at org.apache.beam.sdk.coders.Coder.encode(Coder.java:136)
> at
> org.apache.beam.sdk.util.CoderUtils.encodeToSafeStream(CoderUtils.java:85)
> at
> org.apache.beam.sdk.util.CoderUtils.encodeToByteArray(CoderUtils.java:69)
> at
> org.apache.beam.sdk.util.CoderUtils.encodeToByteArray(CoderUtils.java:54)
> at
> org.apache.beam.sdk.transforms.Create$Values$CreateSource.fromIterable(Create.java:413)
> at org.apache.beam.sdk.transforms.Create$Values.expand(Create.java:370)
> at org.apache.beam.sdk.transforms.Create$Values.expand(Create.java:277)
> at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:548)
> at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:482)
> at org.apache.beam.sdk.values.PBegin.apply(PBegin.java:44)
> at org.apache.beam.sdk.Pipeline.apply(Pipeline.java:177)
> at org.whatever.testing.App.main(App.java:24)
>
>
> package org.whatever.testing;
>
> import com.google.auto.value.AutoValue;
> import com.google.auto.value.extension.memoized.Memoized;
> import org.apache.beam.sdk.Pipeline;
> import org.apache.beam.sdk.options.PipelineOptionsFactory;
> import org.apache.beam.sdk.schemas.AutoValueSchema;
> import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
> import org.apache.beam.sdk.schemas.annotations.SchemaCreate;
> import org.apache.beam.sdk.schemas.transforms.Convert;
> import org.apache.beam.sdk.transforms.Create;
> import org.apache.beam.sdk.transforms.MapElements;
> import org.apache.beam.sdk.values.TypeDescriptor;
>
> import java.util.Arrays;
>
> public class App {
>
>     public static void main(String[] args) {
>         var options = PipelineOptionsFactory.fromArgs(args).withValidation().create();
>
>         var p = Pipeline.create(options);
>         p
>                 .apply(Create.of(Arrays.asList(FooAutoValue.builder().setDummyProp("dummy").build())))
>                 .apply(Convert.to(FooAutoValue.class))
>                 .apply(MapElements.into(TypeDescriptor.of(FooAutoValue.class)).via(i -> {
>                     System.out.println(i.toString());
>                     return i;
>                 }))
>         ;
>         p.run().waitUntilFinish();
>     }
>
>     @AutoValue
>     @DefaultSchema(AutoValueSchema.class)
>     public static abstract class FooAutoValue {
>         public abstract String getDummyProp();
>
>         @Memoized // <-- commenting this line makes everything work
>         public String getSomething(){
>             return "sldj";
>         }
>
>         @SchemaCreate
>         public static FooAutoValue create(String dummyProp) {
>             return builder()
>                     .setDummyProp(dummyProp)
>                     .build();
>         }
>
>         public static Builder builder() {
>             return new AutoValue_App_FooAutoValue.Builder();
>         }
>
>         @AutoValue.Builder
>         public abstract static class Builder {
>             public abstract Builder setDummyProp(String newDummyProp);
>
>             public abstract FooAutoValue build();
>         }
>     }
> }
>
>
> From what I can see, instead of getting the fields from the abstract
> class, it's trying to get the fields from the concrete implementation.
> Given that they are not abstract (as it's custom for AutoValue classes, no
> fields are fetched). This code is in AutoValueSchema.get method.
>
> If confirmed, please create a jira, and I'll gladly look into it, along
> with all the other potential bugs I have sent out recently.
>
> Thank you,
> Cristian
>

Re: Potential bug: AutoValue + Memoized fields

Posted by Luke Cwik <lc...@google.com>.
Thanks for the bug report, I filed
https://issues.apache.org/jira/browse/BEAM-13151

You might be interested in https://issues.apache.org/jira/browse/BEAM-10509
as well.

I couldn't find an obvious JIRA user id based upon the name, do you have a
JIRA user id already?
If not, please create one and share it and I can add you as a contributor
to the BEAM project allowing you to assign issues to yourself. Note that
you can always create issues for Apache projects, all you need is the JIRA
account which anyone can sign up for.



On Thu, Oct 28, 2021 at 11:10 AM Cristian Constantinescu <ze...@gmail.com>
wrote:

> Hi everyone,
>
> Looks like Beam has a little bit of an issue when using AutoValues with
> Memoized (cached) fields. It's not a big issue, and the workaround is
> simply not using Memoised fields at the cost of a little performance. (See
> comment in code snippet)
>
> The code further below produces this exception:
> Exception in thread "main" java.lang.NullPointerException
> at
> org.apache.beam.sdk.schemas.utils.JavaBeanUtils.createGetter(JavaBeanUtils.java:155)
> at
> org.apache.beam.sdk.schemas.utils.JavaBeanUtils.lambda$getGetters$1(JavaBeanUtils.java:145)
> at
> java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
> at
> java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655)
> at
> java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)
> at
> java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
> at
> java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
> at
> java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
> at
> java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
> at
> org.apache.beam.sdk.schemas.utils.JavaBeanUtils.lambda$getGetters$2(JavaBeanUtils.java:146)
> at
> java.base/java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1705)
> at
> org.apache.beam.sdk.schemas.utils.JavaBeanUtils.getGetters(JavaBeanUtils.java:140)
> at
> org.apache.beam.sdk.schemas.AutoValueSchema.fieldValueGetters(AutoValueSchema.java:72)
> at
> org.apache.beam.sdk.schemas.CachingFactory.create(CachingFactory.java:56)
> at org.apache.beam.sdk.values.RowWithGetters.<init>(RowWithGetters.java:66)
> at
> org.apache.beam.sdk.values.Row$Builder.withFieldValueGetters(Row.java:835)
> at
> org.apache.beam.sdk.schemas.GetterBasedSchemaProvider$ToRowWithValueGetters.apply(GetterBasedSchemaProvider.java:64)
> at
> org.apache.beam.sdk.schemas.GetterBasedSchemaProvider$ToRowWithValueGetters.apply(GetterBasedSchemaProvider.java:49)
> at org.apache.beam.sdk.schemas.SchemaCoder.encode(SchemaCoder.java:124)
> at org.apache.beam.sdk.coders.Coder.encode(Coder.java:136)
> at
> org.apache.beam.sdk.util.CoderUtils.encodeToSafeStream(CoderUtils.java:85)
> at
> org.apache.beam.sdk.util.CoderUtils.encodeToByteArray(CoderUtils.java:69)
> at
> org.apache.beam.sdk.util.CoderUtils.encodeToByteArray(CoderUtils.java:54)
> at
> org.apache.beam.sdk.transforms.Create$Values$CreateSource.fromIterable(Create.java:413)
> at org.apache.beam.sdk.transforms.Create$Values.expand(Create.java:370)
> at org.apache.beam.sdk.transforms.Create$Values.expand(Create.java:277)
> at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:548)
> at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:482)
> at org.apache.beam.sdk.values.PBegin.apply(PBegin.java:44)
> at org.apache.beam.sdk.Pipeline.apply(Pipeline.java:177)
> at org.whatever.testing.App.main(App.java:24)
>
>
> package org.whatever.testing;
>
> import com.google.auto.value.AutoValue;
> import com.google.auto.value.extension.memoized.Memoized;
> import org.apache.beam.sdk.Pipeline;
> import org.apache.beam.sdk.options.PipelineOptionsFactory;
> import org.apache.beam.sdk.schemas.AutoValueSchema;
> import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
> import org.apache.beam.sdk.schemas.annotations.SchemaCreate;
> import org.apache.beam.sdk.schemas.transforms.Convert;
> import org.apache.beam.sdk.transforms.Create;
> import org.apache.beam.sdk.transforms.MapElements;
> import org.apache.beam.sdk.values.TypeDescriptor;
>
> import java.util.Arrays;
>
> public class App {
>
>     public static void main(String[] args) {
>         var options = PipelineOptionsFactory.fromArgs(args).withValidation().create();
>
>         var p = Pipeline.create(options);
>         p
>                 .apply(Create.of(Arrays.asList(FooAutoValue.builder().setDummyProp("dummy").build())))
>                 .apply(Convert.to(FooAutoValue.class))
>                 .apply(MapElements.into(TypeDescriptor.of(FooAutoValue.class)).via(i -> {
>                     System.out.println(i.toString());
>                     return i;
>                 }))
>         ;
>         p.run().waitUntilFinish();
>     }
>
>     @AutoValue
>     @DefaultSchema(AutoValueSchema.class)
>     public static abstract class FooAutoValue {
>         public abstract String getDummyProp();
>
>         @Memoized // <-- commenting this line makes everything work
>         public String getSomething(){
>             return "sldj";
>         }
>
>         @SchemaCreate
>         public static FooAutoValue create(String dummyProp) {
>             return builder()
>                     .setDummyProp(dummyProp)
>                     .build();
>         }
>
>         public static Builder builder() {
>             return new AutoValue_App_FooAutoValue.Builder();
>         }
>
>         @AutoValue.Builder
>         public abstract static class Builder {
>             public abstract Builder setDummyProp(String newDummyProp);
>
>             public abstract FooAutoValue build();
>         }
>     }
> }
>
>
> From what I can see, instead of getting the fields from the abstract
> class, it's trying to get the fields from the concrete implementation.
> Given that they are not abstract (as it's custom for AutoValue classes, no
> fields are fetched). This code is in AutoValueSchema.get method.
>
> If confirmed, please create a jira, and I'll gladly look into it, along
> with all the other potential bugs I have sent out recently.
>
> Thank you,
> Cristian
>