-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-25260][SQL] Fix namespace handling in SchemaConverters.toAvroType #22251
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1082,7 +1082,6 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { | |
| val schema = getAvroSchemaStringFromFiles(dir.toString) | ||
| assert(schema.contains("\"namespace\":\"topLevelRecord\"")) | ||
| assert(schema.contains("\"namespace\":\"topLevelRecord.data\"")) | ||
| assert(schema.contains("\"namespace\":\"topLevelRecord.data.data\"")) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -1099,6 +1098,47 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { | |
| } | ||
| } | ||
|
|
||
| test("check namespace - toAvroType") { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @arunmahadevan, can we add a simple end-to-end test as well?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Its sort of covered in the below existing cases. Do you think we need more? Validate namespace in avro file that has nested records with the same name |
||
| val sparkSchema = StructType(Seq( | ||
| StructField("name", StringType, nullable = false), | ||
| StructField("address", StructType(Seq( | ||
| StructField("city", StringType, nullable = false), | ||
| StructField("state", StringType, nullable = false))), | ||
| nullable = false))) | ||
| val employeeType = SchemaConverters.toAvroType(sparkSchema, | ||
| recordName = "employee", | ||
| nameSpace = "foo.bar") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: could you also add a case for
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added a test case for toAvroType with empty namespace |
||
|
|
||
| assert(employeeType.getFullName == "foo.bar.employee") | ||
| assert(employeeType.getName == "employee") | ||
| assert(employeeType.getNamespace == "foo.bar") | ||
|
|
||
| val addressType = employeeType.getField("address").schema() | ||
| assert(addressType.getFullName == "foo.bar.employee.address") | ||
| assert(addressType.getName == "address") | ||
| assert(addressType.getNamespace == "foo.bar.employee") | ||
| } | ||
|
|
||
| test("check empty namespace - toAvroType") { | ||
| val sparkSchema = StructType(Seq( | ||
| StructField("name", StringType, nullable = false), | ||
| StructField("address", StructType(Seq( | ||
| StructField("city", StringType, nullable = false), | ||
| StructField("state", StringType, nullable = false))), | ||
| nullable = false))) | ||
| val employeeType = SchemaConverters.toAvroType(sparkSchema, | ||
| recordName = "employee") | ||
|
|
||
| assert(employeeType.getFullName == "employee") | ||
| assert(employeeType.getName == "employee") | ||
| assert(employeeType.getNamespace == null) | ||
|
|
||
| val addressType = employeeType.getField("address").schema() | ||
| assert(addressType.getFullName == "employee.address") | ||
| assert(addressType.getName == "address") | ||
| assert(addressType.getNamespace == "employee") | ||
| } | ||
|
|
||
| case class NestedMiddleArray(id: Int, data: Array[NestedBottom]) | ||
|
|
||
| case class NestedTopArray(id: Int, data: NestedMiddleArray) | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1, this line is the only difference for the whole code change. The namespace here should not be the one with
recordNameat the end.