From 10a79b728f915e4e04329e80dd6b346d8256142f Mon Sep 17 00:00:00 2001 From: Prabodh Agarwal Date: Thu, 12 Sep 2024 07:50:55 +0530 Subject: [PATCH] pass the avro exception for better information --- .../org/apache/hudi/AvroConversionUtils.scala | 4 ++-- .../apache/hudi/TestAvroConversionUtils.scala | 23 ++++++++++++++----- 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala index cd75da3bb5dac..a4d506be86e78 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala @@ -22,10 +22,9 @@ import org.apache.hudi.HoodieSparkUtils.sparkAdapter import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.exception.SchemaCompatibilityException import org.apache.hudi.internal.schema.HoodieSchemaException - import org.apache.avro.Schema.Type import org.apache.avro.generic.GenericRecord -import org.apache.avro.{JsonProperties, Schema} +import org.apache.avro.{AvroRuntimeException, JsonProperties, Schema} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} @@ -149,6 +148,7 @@ object AvroConversionUtils { val avroSchema = schemaConverters.toAvroType(structType, nullable = false, structName, recordNamespace) getAvroSchemaWithDefaults(avroSchema, structType) } catch { + case a: AvroRuntimeException => throw new HoodieSchemaException(a.getMessage, a) case e: Exception => throw new HoodieSchemaException("Failed to convert struct type to avro schema: " + structType, e) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala index 01de5013b8291..b53750d268251 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala @@ -20,6 +20,7 @@ package org.apache.hudi import org.apache.avro.Schema import org.apache.avro.generic.GenericData +import org.apache.hudi.internal.schema.HoodieSchemaException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types.{ArrayType, BinaryType, DataType, DataTypes, MapType, StringType, StructField, StructType} @@ -182,17 +183,18 @@ class TestAvroConversionUtils extends FunSuite with Matchers { test("test convertStructTypeToAvroSchema with Nested StructField comment") { val mapType = DataTypes.createMapType(StringType, new StructType().add("mapKey", "string", false, "mapKeyComment").add("mapVal", "integer", true)) - val arrayType = ArrayType(new StructType().add("arrayKey", "string", false).add("arrayVal", "integer", true, "arrayValComment")) - val innerStruct = new StructType().add("innerKey","string",false, "innerKeyComment").add("value", "long", true, "valueComment") + val arrayType = ArrayType(new StructType().add("arrayKey", "string", false).add("arrayVal", "integer", true, "arrayValComment")) + val innerStruct = new StructType().add("innerKey", "string", false, "innerKeyComment").add("value", "long", true, "valueComment") val struct = new StructType().add("key", "string", false).add("version", "string", true, "versionComment") - .add("data1",innerStruct,false).add("data2",innerStruct,true) - .add("nullableMap", mapType, true).add("map",mapType,false) - .add("nullableArray", arrayType, true).add("array",arrayType,false) + .add("data1", innerStruct, false).add("data2", innerStruct, true) + .add("nullableMap", mapType, true).add("map", mapType, false) + .add("nullableArray", arrayType, true).add("array", arrayType, false) val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(struct, "SchemaName", "SchemaNS") - val expectedSchemaStr = s""" + val expectedSchemaStr = + s""" { "type": "record", "name": "SchemaName", @@ -445,4 +447,13 @@ class TestAvroConversionUtils extends FunSuite with Matchers { private def checkNull(left: Any, right: Any): Boolean = { (left == null && right != null) || (left == null && right != null) } + + test("convert struct type with duplicate column names") { + val struct = new StructType().add("id", DataTypes.LongType, true) + .add("name", DataTypes.StringType, true) + .add("name", DataTypes.StringType, true) + the[HoodieSchemaException] thrownBy { + AvroConversionUtils.convertStructTypeToAvroSchema(struct, "SchemaName", "SchemaNS") + } should have message "Duplicate field name in record SchemaNS.SchemaName: name type:UNION pos:2 and name type:UNION pos:1." + } }