You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/06/04 20:43:32 UTC

[GitHub] [beam] damccorm opened a new issue, #20954: Schema from JavaBeanSchema causes stack underflow

damccorm opened a new issue, #20954:
URL: https://github.com/apache/beam/issues/20954

   Minimum example: [https://github.com/akettmann-e24/beamErrorExample](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:
   ```
   
   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
   
   ```
   
    
   
   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.
   
   Imported from Jira [BEAM-12360](https://issues.apache.org/jira/browse/BEAM-12360). Original Jira may contain additional context.
   Reported by: akettmann.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] reuvenlax commented on issue #20954: Schema from JavaBeanSchema causes stack underflow

Posted by GitBox <gi...@apache.org>.
reuvenlax commented on issue #20954:
URL: https://github.com/apache/beam/issues/20954#issuecomment-1306051285

   I think we'll have to add explicit support for the Java 8 types - Beam historically has used Joda types, so we never added Java 8 support here.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit commented on issue #20954: Schema from JavaBeanSchema causes stack underflow

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on issue #20954:
URL: https://github.com/apache/beam/issues/20954#issuecomment-1299177796

   Some comments from the jira.
   
   From Andrew Kettman:
   > Appears that using a `org.joda.time.Instant` instead of the built in Java Instant may be a workaround.
   
   From Jonathan Kay:
   > This is still an issue as of beam version 2.35.0.
   >
   > Not limited to Instant, the issue also occurs with other Java 8+ date types (LocalDateTime, ZonedDateTime, etc). Joda types result in a similar error.
   > 
   > Are there any workarounds allowing schema inference for classes containing Java (or even Joda) date types?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] TheNeuralBit commented on issue #20954: Schema from JavaBeanSchema causes stack underflow

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on issue #20954:
URL: https://github.com/apache/beam/issues/20954#issuecomment-1299178590

   @reuvenlax may have some insights


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org