xkrogen commented on code in PR #36506:
URL: https://github.com/apache/spark/pull/36506#discussion_r964068530


##########
connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala:
##########
@@ -287,14 +298,51 @@ private[sql] class AvroSerializer(
       result
   }
 
+  /**
+   * Complex unions map to struct types where field names are member0, 
member1, etc.
+   * This is consistent with the behavior in [[SchemaConverters]] and when 
converting between Avro
+   * and Parquet.
+   */
+  private def newComplexUnionConverter(
+      catalystStruct: StructType,
+      unionType: Schema,
+      catalystPath: Seq[String],
+      avroPath: Seq[String]): InternalRow => Any = {
+    val nonNullTypes = nonNullUnionBranches(unionType)
+    val expectedFieldNames = nonNullTypes.indices.map(i => s"member$i")
+    if (catalystStruct.fieldNames.toSeq != expectedFieldNames) {
+      throw new IncompatibleSchemaException(s"Generic Avro union at 
${toFieldStr(avroPath)} " +
+        s"does not match the SQL schema at ${toFieldStr(catalystPath)}.  It 
expected the " +
+        s"following members ${expectedFieldNames.mkString("(", ", ", ")")} but 
got " +
+        s"${catalystStruct.fieldNames.mkString("(", ", ", ")")}")
+    }

Review Comment:
   We should respect `AvroOptions.positionalFieldMatching` here and only do 
this check if `positionalFieldMatching = false`, WDYT?



##########
connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala:
##########
@@ -323,15 +329,30 @@ abstract class AvroSuite
       avroRec.put("field2", field2)
       avroRec.put("field3", new Fixed(fixedSchema, field3))
       avroRec.put("field4", new EnumSymbol(enumSchema, field4))
+      avroRec.put("field5", null)
       dataFileWriter.append(avroRec)
       dataFileWriter.flush()
       dataFileWriter.close()
 
-      val df = spark.sqlContext.read.format("avro").load(s"$dir.avro")
+      val df = spark.sqlContext.read.format("avro").load(nativeWriterPath)
       assertResult(field1)(df.selectExpr("field1.member0").first().get(0))
       assertResult(field2)(df.selectExpr("field2.member1").first().get(0))
       assertResult(field3)(df.selectExpr("field3.member2").first().get(0))
       assertResult(field4)(df.selectExpr("field4.member3").first().get(0))
+
+      df.write.format("avro").option("avroSchema", 
schema.toString).save(sparkWriterPath)
+
+      val df2 = spark.sqlContext.read.format("avro").load(nativeWriterPath)
+      assertResult(field1)(df2.selectExpr("field1.member0").first().get(0))
+      assertResult(field2)(df2.selectExpr("field2.member1").first().get(0))
+      assertResult(field3)(df2.selectExpr("field3.member2").first().get(0))
+      assertResult(field4)(df2.selectExpr("field4.member3").first().get(0))

Review Comment:
   Can we confirm the value of `df2.selectExpr("field5")` as well (here and 
above)?



##########
connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala:
##########
@@ -219,7 +219,7 @@ class AvroFunctionsSuite extends QueryTest with 
SharedSparkSession {
       functions.from_avro($"avro", avroTypeStruct)), df)
   }
 
-  test("to_avro with unsupported nullable Avro schema") {
+  test("to_avro with complex union Avro schema") {
     val df = spark.range(10).select(struct($"id", 
$"id".cast("string").as("str")).as("struct"))
     for (unsupportedAvroType <- Seq("""["null", "int", "long"]""", """["int", 
"long"]""")) {

Review Comment:
   Yeah I agree the test as-is won't work. I was thinking more like adding 
another test for the more complex case. My point was more that the 
int-long/float-double case is only ... "partially complex"? And this test 
claims to be testing the complex case, which (to me at least) would indicate 
that we get an exploded record. 



##########
connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala:
##########
@@ -287,14 +298,54 @@ private[sql] class AvroSerializer(
       result
   }
 
+  /**
+   * Complex unions map to struct types where field names are member0, 
member1, etc.
+   * This is consistent with the behavior in [[SchemaConverters]] and when 
converting between Avro
+   * and Parquet.
+   */
+  private def newComplexUnionConverter(
+      catalystStruct: StructType,
+      avroType: Schema,
+      catalystPath: Seq[String],
+      avroPath: Seq[String]): InternalRow => Any = {
+    val nonNullTypes = avroType.getTypes.asScala.filter(_.getType != 
NULL).toSeq
+    validateComplexUnionMembers(catalystStruct, nonNullTypes, catalystPath, 
avroPath)
+
+    val fieldConverters = nonNullTypes.zipWithIndex.map { case (avroField, i) 
=>
+      val cf = catalystStruct.fields(i)
+      newConverter(cf.dataType, resolveNullableType(avroField, nullable = 
true),
+        catalystPath :+ cf.name, avroPath :+ cf.name)

Review Comment:
   Yeah that's fair. Slightly less precise, but I agree it will probably be 
more in-line with user expectations.



-- 
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]

Reply via email to