Skip to content

Conversation

@gengliangwang
Copy link
Member

What changes were proposed in this pull request?

Avro schema allows recursive reference, e.g. the schema for linked-list in https://avro.apache.org/docs/1.8.2/spec.html#schema_record

{
  "type": "record",
  "name": "LongList",
  "aliases": ["LinkedLongs"],                      // old name for this
  "fields" : [
    {"name": "value", "type": "long"},             // each element has a long
    {"name": "next", "type": ["null", "LongList"]} // optional next element
  ]
}

In current Spark SQL, it is impossible to convert the schema as StructType . Run SchemaConverters.toSqlType(avroSchema) and we will get stack overflow exception.

We should detect the recursive reference and throw exception for it.

How was this patch tested?

New unit test case.

@SparkQA
Copy link

SparkQA commented Oct 12, 2018

Test build #97310 has finished for PR 22709 at commit c97f543.

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

case ENUM => SchemaType(StringType, nullable = false)

case RECORD =>
if (existingRecordNames.contains(avroSchema.getFullName)) {
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Another approach is to check the whole json string schema(avroSchema.toString) here. But it seems overkill. Avro requires the full name of record to be unique.

@cloud-fan
Copy link
Contributor

LGTM, merging to master/2.4!

asfgit pushed a commit that referenced this pull request Oct 13, 2018
…w exception

## What changes were proposed in this pull request?

Avro schema allows recursive reference, e.g. the schema for linked-list in https://avro.apache.org/docs/1.8.2/spec.html#schema_record
```
{
  "type": "record",
  "name": "LongList",
  "aliases": ["LinkedLongs"],                      // old name for this
  "fields" : [
    {"name": "value", "type": "long"},             // each element has a long
    {"name": "next", "type": ["null", "LongList"]} // optional next element
  ]
}
```

In current Spark SQL, it is impossible to convert the schema as `StructType` . Run `SchemaConverters.toSqlType(avroSchema)` and we will get stack overflow exception.

We should detect the recursive reference and throw exception for it.
## How was this patch tested?

New unit test case.

Closes #22709 from gengliangwang/avroRecursiveRef.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 2eaf058)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
@asfgit asfgit closed this in 2eaf058 Oct 13, 2018
jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
…w exception

## What changes were proposed in this pull request?

Avro schema allows recursive reference, e.g. the schema for linked-list in https://avro.apache.org/docs/1.8.2/spec.html#schema_record
```
{
  "type": "record",
  "name": "LongList",
  "aliases": ["LinkedLongs"],                      // old name for this
  "fields" : [
    {"name": "value", "type": "long"},             // each element has a long
    {"name": "next", "type": ["null", "LongList"]} // optional next element
  ]
}
```

In current Spark SQL, it is impossible to convert the schema as `StructType` . Run `SchemaConverters.toSqlType(avroSchema)` and we will get stack overflow exception.

We should detect the recursive reference and throw exception for it.
## How was this patch tested?

New unit test case.

Closes apache#22709 from gengliangwang/avroRecursiveRef.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
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.

3 participants