Andrew Kettmann created BEAM-12360:
--------------------------------------
Summary: Schema from JavaBeanSchema causes stack underflow
Key: BEAM-12360
URL: https://issues.apache.org/jira/browse/BEAM-12360
Project: Beam
Issue Type: Bug
Components: sdk-java-core
Affects Versions: 2.29.0, 2.28.0
Reporter: Andrew Kettmann
Minimum example: [https://github.com/akettmann-e24/beamErrorExample]
Runner: DirectRunner
Tested under JVMs:
* AdoptJDK Java 11.0.10
* AdoptJDK Java 11.0.11
* Oracle Java 1.8.0.262
Tested under beam version:
* 2.28.0
* 2.29.0
When using a schema with a JavaBeanSchema, if a field is an
`Java.time.Instant`, pipeline fails once the value has been returned and the
return value needs to be encoded into a ByteArray. Stack trace below:
{code:java}
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in
[jar:file:/home/akettmann/.m2/repository/ch/qos/logback/logback-classic/1.2.3/logback-classic-1.2.3.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in
[jar:file:/home/akettmann/.m2/repository/org/slf4j/slf4j-log4j12/1.7.25/slf4j-log4j12-1.7.25.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
SLF4J: Actual binding is of type
[ch.qos.logback.classic.util.ContextSelectorStaticBinder]
14:20:53.698 [main] DEBUG org.apache.beam.sdk.options.PipelineOptionsFactory -
Provided Arguments: {}
14:20:53.752 [main] DEBUG org.apache.beam.sdk.Pipeline - Creating
Pipeline#998062648
14:20:53.758 [main] DEBUG org.apache.beam.sdk.Pipeline - Adding Create.Values
to Pipeline#998062648
Exception in thread "main" java.lang.VerifyError: Operand stack underflow
Exception Details:
Location:
org/apache/beam/sdk/schemas/FieldValueGetter$SchemaCodeGen$tyVybsgB.get(Ljava/time/Instant;)Ljava/lang/Long;
@1: invokevirtual
Reason:
Attempt to pop empty stack.
Current Frame:
bci: @1
flags: { }
locals: {
'org/apache/beam/sdk/schemas/FieldValueGetter$SchemaCodeGen$tyVybsgB',
'java/time/Instant' }
stack: { 'java/time/Instant' }
Bytecode:
0000000: 2bb6 0016 b800 1cb0
at java.base/java.lang.Class.getDeclaredFields0(Native Method)
at java.base/java.lang.Class.privateGetDeclaredFields(Class.java:3061)
at java.base/java.lang.Class.getDeclaredField(Class.java:2409)
at
org.apache.beam.vendor.bytebuddy.v1_10_8.net.bytebuddy.implementation.LoadedTypeInitializer$ForStaticField.onLoad(LoadedTypeInitializer.java:122)
at
org.apache.beam.vendor.bytebuddy.v1_10_8.net.bytebuddy.implementation.LoadedTypeInitializer$Compound.onLoad(LoadedTypeInitializer.java:192)
at
org.apache.beam.vendor.bytebuddy.v1_10_8.net.bytebuddy.dynamic.TypeResolutionStrategy$Passive.initialize(TypeResolutionStrategy.java:102)
at
org.apache.beam.vendor.bytebuddy.v1_10_8.net.bytebuddy.dynamic.DynamicType$Default$Unloaded.load(DynamicType.java:5707)
at
org.apache.beam.sdk.schemas.utils.JavaBeanUtils.createGetter(JavaBeanUtils.java:162)
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.JavaBeanSchema.fieldValueGetters(JavaBeanSchema.java:142)
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.RowWithGetters.getValue(RowWithGetters.java:111)
at org.apache.beam.sdk.values.RowWithGetters.getValue(RowWithGetters.java:78)
at
org.apache.beam.sdk.coders.RowCoderGenerator$EncodeInstruction.encodeDelegate(RowCoderGenerator.java:272)
at org.apache.beam.sdk.coders.Coder$ByteBuddy$xu4m7ooW.encode(Unknown Source)
at org.apache.beam.sdk.coders.Coder$ByteBuddy$xu4m7ooW.encode(Unknown Source)
at org.apache.beam.sdk.schemas.SchemaCoder.encode(SchemaCoder.java:118)
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:547)
at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:481)
at org.apache.beam.sdk.values.PBegin.apply(PBegin.java:44)
at org.apache.beam.sdk.Pipeline.apply(Pipeline.java:176)
at
com.example.beam.coder.BeanSchemaErrorExample.run(BeanSchemaErrorExample.java:28)
at
com.example.beam.coder.BeanSchemaErrorExample.main(BeanSchemaErrorExample.java:19)
Disconnected from the target VM, address: '127.0.0.1:42807', transport: 'socket'
Process finished with exit code 1
{code}
Slightly different stack trace when it is from a ParDo returning a type that
has a schema inferred, but the same end result of a stack underflow.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)