[ 
https://issues.apache.org/jira/browse/AVRO-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated AVRO-4039:
---------------------------------
    Labels: pull-request-available  (was: )

> ClassCastException when deserializing an array of logical types
> ---------------------------------------------------------------
>
>                 Key: AVRO-4039
>                 URL: https://issues.apache.org/jira/browse/AVRO-4039
>             Project: Apache Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.12.0
>            Reporter: William McNutt
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> With a schema
> {code:java}
> {"type":"record","name":"Root","fields":[
> {"name":"f_date_repeated","type":{"type":"array","items": 
> {"type":"int","logicalType":"date"}}},
> {"name":"f_time_repeated","type":{"type":"array","items":{"type":"long","logicalType":"time-micros"}}},
> {"name":"f_datetime_repeated","type":{"type":"array","items":{"type":"string","logicalType":"datetime"}}}
> ]}
> {code}
> and converters added:
> {code:java}
> static{ 
> GenericData.get().addLogicalTypeConversion(new 
> Conversions.DecimalConversion());
> GenericData.get().addLogicalTypeConversion(new 
> TimeConversions.DateConversion());
> GenericData.get().addLogicalTypeConversion(new 
> TimeConversions.TimeMillisConversion());
> GenericData.get().addLogicalTypeConversion(new 
> TimeConversions.TimeMicrosConversion());
> GenericData.get().addLogicalTypeConversion(new 
> TimeConversions.TimestampMillisConversion());
> GenericData.get().addLogicalTypeConversion(new 
> TimeConversions.TimestampMicrosConversion());
> GenericData.get().addLogicalTypeConversion(new 
> TimeConversions.LocalTimestampMicrosConversion());
> GenericData.get().addLogicalTypeConversion(new 
> TimeConversions.LocalTimestampMillisConversion());
> }
> {code}
> we are get an exception (and similarly Instant cannot be cast to Long)
> {code:java}
> ava.lang.ClassCastException: class java.time.LocalDate cannot be cast to 
> class java.lang.Integer (java.time.LocalDate and java.lang.Integer are in 
> module java.base of loader 'bootstrap')
> at 
> org.apache.avro.generic.PrimitivesArrays$IntArray.add(PrimitivesArrays.java:28)
> at java.base/java.util.AbstractList.add(Unknown Source)
> at 
> org.apache.avro.generic.GenericDatumReader.addToArray(GenericDatumReader.java:333)
> at 
> org.apache.avro.generic.GenericDatumReader.readArray(GenericDatumReader.java:294)
> at 
> org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:184)
> at 
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:161)
> at 
> org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:260)
> at 
> org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:248)
> at 
> org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:180)
> at 
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:161)
> at 
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:154)
> {code}
> When the fields are instead an optional or required rather than an array, 
> there is no exception thrown.
> Testing locally, I think 
> [https://github.com/apache/avro/blob/885c62d5af9df6813b6ed20ae30f785b756243e3/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java#L1531-L1547]
>  is the issue.
> Adding
> {code:java}
> if (schema.getElementType().getLogicalType() != null) {
>   return new GenericData.Array<Object>(size, schema);
> }
> {code}
> fixes it:
> {code:java}
> public Object newArray(Object old, int size, Schema schema) {
>     if (old instanceof GenericArray) {
>       ((GenericArray<?>) old).reset();
>       return old;
>     } else if (old instanceof Collection) {
>       ((Collection<?>) old).clear();
>       return old;
>     } else {
>       if (schema.getElementType().getLogicalType() != null) {
>         return new GenericData.Array<Object>(size, schema);
>       }
>       if (schema.getElementType().getType() == Type.INT) {
>         return new PrimitivesArrays.IntArray(size, schema);
>       }
>       if (schema.getElementType().getType() == Type.BOOLEAN) {
>         return new PrimitivesArrays.BooleanArray(size, schema);
>       }
>       if (schema.getElementType().getType() == Type.LONG) {
>         return new PrimitivesArrays.LongArray(size, schema);
>       }
>       if (schema.getElementType().getType() == Type.FLOAT) {
>         return new PrimitivesArrays.FloatArray(size, schema);
>       }
>       if (schema.getElementType().getType() == Type.DOUBLE) {
>         return new PrimitivesArrays.DoubleArray(size, schema);
>       }
>       return new GenericData.Array<Object>(size, schema);
>     }
>   }
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to