[
https://issues.apache.org/jira/browse/AVRO-4203?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18038469#comment-18038469
]
Yi Hu commented on AVRO-4203:
-----------------------------
A simple, but very specific fix would be return long/double field index for
UnionSchema.getIndexNamed(
name):
[https://github.com/apache/avro/blob/6b94ef87b6ead692f276c6aabf0c7ebf28daaa51/lang/java/avro/src/main/java/org/apache/avro/Schema.java#L1288]
do something like
{code:java}
@Override
public Integer getIndexNamed(String name) {
Integer retval = indexByName.get(name);
if (retval == null) {
// allowed casting: int->long; float->double
if(name.equals("int")) {
return indexByName.get("long");
} else if (name.equals("float")) {
return indexByName.get("double");
}
}
return retval;
} {code}
> inconsistent implicit int->long conversion for non-null and nullable long
> -------------------------------------------------------------------------
>
> Key: AVRO-4203
> URL: https://issues.apache.org/jira/browse/AVRO-4203
> Project: Apache Avro
> Issue Type: Bug
> Components: java
> Reporter: Yi Hu
> Priority: Major
>
> When using GenericDatumWriter to write an int to a long field, inconsistent
> implicit int->long conversion is observed depending on the type is nullable
> or not:
>
> * write int to target schema int64, succeed
> * write int to target schema nullable(int64), using storage_write_api, succeed
> * write int to target schema nullable(int64), using file_load avro, exception
>
> An example that reproduces the issue:
>
> {code:java}
> public class AvroTest {
> private static final String SCHEMA_JSON = "{\n" +
> " \"type\": \"record\",\n" +
> " \"name\": \"UserEvent\",\n" +
> " \"namespace\": \"com.example.avro\",\n" +
> " \"fields\": [\n" +
> " {\"name\": \"userId\", \"type\": \"string\"},\n" +
> " {\"name\": \"nonNullLong\", \"type\": \"long\"},\n" +
> " {\"name\": \"nullableLong\", \"type\": [\"null\", \"long\"],
> \"default\": null}\n" +
> " ]\n" +
> "}"; public static void main(String[] argv) throws
> AvroRuntimeException, IOException {
> Schema schema = new Schema.Parser().parse(SCHEMA_JSON);
> GenericRecord eventWithTimestamp = new GenericData.Record(schema);
> eventWithTimestamp.put("userId", "user-123");
> eventWithTimestamp.put("nonNullLong", 123);
> eventWithTimestamp.put("nullableLong", 123); // fail
> File avroOutputFile = new File("user-events.avro");
> DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
> try (DataFileWriter<GenericRecord> dataFileWriter = new
> DataFileWriter<>(datumWriter)) {
> dataFileWriter.create(schema, avroOutputFile);
> dataFileWriter.append(eventWithTimestamp);
> }
> }
> } {code}
>
> Stack trace:
>
> {code:java}
> Caused by: org.apache.avro.UnresolvedUnionException: Not in union
> ["null","long"]: 123 (field=nullableLong)
> at
> org.apache.avro.generic.GenericDatumWriter.writeField(GenericDatumWriter.java:247)
> at
> org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:234)
> at
> org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:145)
> at
> org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:95)
> at
> org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:82)
> at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:314)
> ... 1 more
> Suppressed: org.apache.avro.UnresolvedUnionException: Not in union
> ["null","long"]: 123
> Caused by: org.apache.avro.UnresolvedUnionException: Not in union
> ["null","long"]: 123 (field=nullableLong)
> at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:910)
> at
> org.apache.avro.generic.GenericDatumWriter.resolveUnion(GenericDatumWriter.java:307)
> at
> org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:157)
> at
> org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:95)
> at
> org.apache.avro.generic.GenericDatumWriter.writeField(GenericDatumWriter.java:245)
> ... 6 more {code}
> This affects Apache Beam: https://github.com/apache/beam/issues/36735
--
This message was sent by Atlassian Jira
(v8.20.10#820010)