Skip to content

Commit

Permalink
[MINOR] Pass the avro exception for better information (#11981)
Browse files Browse the repository at this point in the history
  • Loading branch information
prabodh1194 authored Sep 22, 2024
1 parent 3883285 commit 818fc3b
Show file tree
Hide file tree
Showing 2 changed files with 12 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ 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}
Expand Down Expand Up @@ -149,6 +149,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)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import java.nio.ByteBuffer
import java.util.Objects
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}
Expand Down Expand Up @@ -444,4 +445,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."
}
}

0 comments on commit 818fc3b

Please sign in to comment.