Luke Cwik created BEAM-13151:
--------------------------------
Summary: AutoValue with @Memoized fields unsupported with Schema
Key: BEAM-13151
URL: https://issues.apache.org/jira/browse/BEAM-13151
Project: Beam
Issue Type: Bug
Components: sdk-java-core
Reporter: Luke Cwik
Sourced from:
https://lists.apache.org/thread.html/rd0c65fca675451f25a6ec02db5df163d1deb6cd6216144d0dbfb0374%40%3Cdev.beam.apache.org%3E
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:
{noformat}
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)
{noformat}
{code:java}
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();
}
}
}
{code}
>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.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)