Skip to content

Conversation

@arunmahadevan
Copy link
Contributor

What changes were proposed in this pull request?

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,

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

How was this patch tested?

New and existing unit tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

@arunmahadevan
Copy link
Contributor Author

cc @gengliangwang @dongjoon-hyun

@SparkQA
Copy link

SparkQA commented Aug 28, 2018

Test build #95338 has finished for PR 22251 at commit f474839.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

}

val childNameSpace = if (nameSpace != "") s"$nameSpace.$recordName" else recordName
val fieldsAssembler = builder.record(recordName).namespace(nameSpace).fields()
Copy link
Member

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") {
Copy link
Member

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?

Copy link
Contributor Author

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")
Copy link
Member

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 "" ?

Copy link
Contributor Author

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

Copy link
Member

@gengliangwang gengliangwang left a 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!

@SparkQA
Copy link

SparkQA commented Aug 28, 2018

Test build #95367 has finished for PR 22251 at commit 2153428.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

Merged to master.

@asfgit asfgit closed this in 68ec207 Aug 29, 2018
otterc pushed a commit to linkedin/spark that referenced this pull request Mar 22, 2023
`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
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants