xkrogen commented on code in PR #36506:
URL: https://github.com/apache/spark/pull/36506#discussion_r960091825
##########
connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala:
##########
@@ -218,6 +218,17 @@ private[sql] class AvroSerializer(
val numFields = st.length
(getter, ordinal) => structConverter(getter.getStruct(ordinal,
numFields))
+ case (st: StructType, UNION) =>
+ val unionConvertor = newComplexUnionConverter(st, avroType,
catalystPath, avroPath)
+ val numFields = st.length
+ (getter, ordinal) => unionConvertor(getter.getStruct(ordinal,
numFields))
+
+ case (DoubleType, UNION) if nonNullUnionTypes(avroType) == Set(FLOAT,
DOUBLE) =>
+ (getter, ordinal) => getter.getDouble(ordinal)
+
+ case (LongType, UNION) if nonNullUnionTypes(avroType) == Set(INT, LONG)
=>
+ (getter, ordinal) => getter.getLong(ordinal)
Review Comment:
> as the schema has to be provided when writing avro
It's not required to provide the schema, right? If you don't provide one,
then we'll use `SchemaConverters.toAvroType()` to derive a schema used at
write-time. But that's not what I'm worried about; that part will always be
lossy since there are things (like unions) that `toAvroType()` will not
generate.
Even if you do provide the same schema used at the input, the values
themselves will all be long/double regardless of their input type.
Consider a record with a field schema like:
```
{ "name": "foo", "type": ["int", "long"] }
```
We have some input dataset that contains two records, one with the value `1`
(int) and one with the value `2` (long). If we read this via Spark, then write
it back _using the same schema_:
```
val avroSchema = ...
val df = spark.read.format("avro").option("avroSchema",
avroSchema).load("/path/to/input.avro")
df.write.format("avro").option("avroSchema",
avroSchema).save("/path/to/output.avro")
```
Now `output.avro` contains two records, with value 1 and 2, except that both
are stored as long (the second union branch). This is different from
`input.avro`, which has one int (first union branch) and one long (second union
branch).
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]