[ https://issues.apache.org/jira/browse/FLINK-18478?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17164261#comment-17164261 ]
Aljoscha Krettek commented on FLINK-18478: ------------------------------------------ So, the reason why it's deserializing the record as a generic record is that it cannot find the {{Tweet}} class. If you put the class in the right Java/Scala package ({{com.github.geoheil.streamingreference}} in your example) you will get one step further. Then it fails with this, though: {code} Exception in thread "main" org.apache.avro.AvroRuntimeException: avro.shaded.com.google.common.util.concurrent.UncheckedExecutionException: org.apache.avro.AvroRuntimeException: Not a Specific class: class org.apache.flink.streaming.scala.examples.windowing.Tweet at org.apache.avro.specific.SpecificData.getSchema(SpecificData.java:227) at org.apache.avro.specific.SpecificDatumReader.setSchema(SpecificDatumReader.java:74) at org.apache.flink.formats.avro.AvroDeserializationSchema.deserialize(AvroDeserializationSchema.java:134) at org.apache.flink.streaming.scala.examples.windowing.AvroBug$.main(WindowWordCount.scala:67) at org.apache.flink.streaming.scala.examples.windowing.AvroBug.main(WindowWordCount.scala) Caused by: avro.shaded.com.google.common.util.concurrent.UncheckedExecutionException: org.apache.avro.AvroRuntimeException: Not a Specific class: class org.apache.flink.streaming.scala.examples.windowing.Tweet at avro.shaded.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2234) at avro.shaded.com.google.common.cache.LocalCache.get(LocalCache.java:3965) at avro.shaded.com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3969) at avro.shaded.com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4829) at org.apache.avro.specific.SpecificData.getSchema(SpecificData.java:225) ... 4 more Caused by: org.apache.avro.AvroRuntimeException: Not a Specific class: class org.apache.flink.streaming.scala.examples.windowing.Tweet at org.apache.avro.specific.SpecificData.createSchema(SpecificData.java:285) at org.apache.avro.specific.SpecificData$2.load(SpecificData.java:218) at org.apache.avro.specific.SpecificData$2.load(SpecificData.java:215) at avro.shaded.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3568) at avro.shaded.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2350) at avro.shaded.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2313) at avro.shaded.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2228) ... 8 more {code} The reason here is that Avro-internal code tries to access the {{SCHEMA$}} field, which is not there for avrohugger-generated classes. The field is in the companion object, but that's not the same as a {{static}} the the class itself, which is what Avro expects. [~dwysakowicz] The problematic call is this in {{AvroDeserializationSchema}}: {code} datumReader.setSchema(readerSchema); {code} maybe we could get around this by creating encoders/serializers/whatnot differently but I'm slowly getting a bit fed-up with avrohugger. > AvroDeserializationSchema does not work with types generated by avrohugger > -------------------------------------------------------------------------- > > Key: FLINK-18478 > URL: https://issues.apache.org/jira/browse/FLINK-18478 > Project: Flink > Issue Type: Bug > Components: Formats (JSON, Avro, Parquet, ORC, SequenceFile) > Reporter: Aljoscha Krettek > Assignee: Aljoscha Krettek > Priority: Major > Labels: pull-request-available > Fix For: 1.10.2, 1.12.0, 1.11.1 > > > The main problem is that the code in {{SpecificData.createSchema()}} tries to > reflectively read the {{SCHEMA$}} field, that is normally there in Avro > generated classes. However, avrohugger generates this field in a companion > object, which the reflective Java code will therefore not find. > This is also described in these ML threads: > * > [https://lists.apache.org/thread.html/5db58c7d15e4e9aaa515f935be3b342fe036e97d32e1fb0f0d1797ee@%3Cuser.flink.apache.org%3E] > * > [https://lists.apache.org/thread.html/cf1c5b8fa7f095739438807de9f2497e04ffe55237c5dea83355112d@%3Cuser.flink.apache.org%3E] -- This message was sent by Atlassian Jira (v8.3.4#803005)