Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[MINOR] pass the avro exception for better information #11981

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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."
}
}
Loading