-
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
Conversation
|
Test build #95338 has finished for PR 22251 at commit
|
| } | ||
|
|
||
| val childNameSpace = if (nameSpace != "") s"$nameSpace.$recordName" else recordName | ||
| val fieldsAssembler = builder.record(recordName).namespace(nameSpace).fields() |
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 recordName at the end.
| } | ||
| } | ||
|
|
||
| test("check namespace - toAvroType") { |
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.
@arunmahadevan, can we add a simple end-to-end test as well?
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.
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
conversion to avro and back with namespace
| nullable = false))) | ||
| val employeeType = SchemaConverters.toAvroType(sparkSchema, | ||
| recordName = "employee", | ||
| nameSpace = "foo.bar") |
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.
nit: could you also add a case for nameSpace as "" ?
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.
Added a test case for toAvroType with empty namespace
gengliangwang
left a comment
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.
LGTM, thanks for the fix!
5330c45 to
2153428
Compare
|
Test build #95367 has finished for PR 22251 at commit
|
|
Merged to master. |
`toAvroType` converts spark data type to avro schema. It always appends the record name to namespace so its impossible to have an Avro namespace independent of the record name.
When invoked with a spark data type like,
```java
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)))
// map it to an avro schema with record name "employee" and top level namespace "foo.bar",
val avroSchema = SchemaConverters.toAvroType(sparkSchema, false, "employee", "foo.bar")
// result is
// avroSchema.getName = employee
// avroSchema.getNamespace = foo.bar.employee
// avroSchema.getFullname = foo.bar.employee.employee
```
The patch proposes to fix this so that the result is
```
avroSchema.getName = employee
avroSchema.getNamespace = foo.bar
avroSchema.getFullname = foo.bar.employee
```
New and existing unit tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes apache#22251 from arunmahadevan/avro-fix.
Authored-by: Arun Mahadevan <[email protected]>
Signed-off-by: hyukjinkwon <[email protected]>
(cherry picked from commit 68ec207)
RB=2106636
BUG=LIHADOOP-53221
G=spark-reviewers
R=mshen,ekrogen
A=ekrogen
What changes were proposed in this pull request?
toAvroTypeconverts spark data type to avro schema. It always appends the record name to namespace so its impossible to have an Avro namespace independent of the record name.When invoked with a spark data type like,
The patch proposes to fix this so that the result is
How was this patch tested?
New and existing unit tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.