-
Notifications
You must be signed in to change notification settings - Fork 28k
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
[SPARK-8811][SQL] Read array struct data from parquet error #7209
Changes from all commits
ecd2547
3d38a75
e887706
49ca112
d931141
0069895
2480abd
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 |
---|---|---|
|
@@ -174,7 +174,7 @@ class ParquetSchemaInferenceSuite extends ParquetSchemaTest { | |
""" | ||
|message root { | ||
| optional group _1 (LIST) { | ||
| repeated int32 element; | ||
| repeated int32 array; | ||
| } | ||
|} | ||
""".stripMargin) | ||
|
@@ -198,7 +198,7 @@ class ParquetSchemaInferenceSuite extends ParquetSchemaTest { | |
|message root { | ||
| optional group _1 (LIST) { | ||
| repeated group bag { | ||
| optional int32 element; | ||
| optional int32 array; | ||
| } | ||
| } | ||
|} | ||
|
@@ -267,7 +267,7 @@ class ParquetSchemaInferenceSuite extends ParquetSchemaTest { | |
| optional binary _1 (UTF8); | ||
| optional group _2 (LIST) { | ||
| repeated group bag { | ||
| optional group element { | ||
| optional group array { | ||
| required int32 _1; | ||
| required double _2; | ||
| } | ||
|
@@ -467,7 +467,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | |
"""message root { | ||
| optional group f1 (LIST) { | ||
| repeated group list { | ||
| optional int32 element; | ||
| optional int32 array; | ||
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. Changes made are limited to Catalyst to Parquet schema conversion, so there's no need to modify this Parquet to Catalyst test case. |
||
| } | ||
| } | ||
|} | ||
|
@@ -478,11 +478,12 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | |
StructType(Seq( | ||
StructField( | ||
"f1", | ||
ArrayType(IntegerType, containsNull = true), | ||
ArrayType( | ||
StructType(Seq(StructField("num", IntegerType))), containsNull = false), | ||
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. Hm, this change is supposed to be wrong. The Parquet schema below just represents an integer array. Don't know why it passed tests... Trying to figure this out. 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. Figured out why the test passes. It's because |
||
nullable = true))), | ||
"""message root { | ||
| optional group f1 (LIST) { | ||
| repeated group element { | ||
| repeated group array { | ||
| optional int32 num; | ||
| } | ||
| } | ||
|
@@ -496,7 +497,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | |
"""message root { | ||
| optional group f1 (LIST) { | ||
| repeated group list { | ||
| required int32 element; | ||
| required int32 array; | ||
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. This change should be unnecessary either. |
||
| } | ||
| } | ||
|} | ||
|
@@ -505,10 +506,13 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | |
testParquetToCatalyst( | ||
"Backwards-compatibility: LIST with non-nullable element type - 2", | ||
StructType(Seq( | ||
StructField("f1", ArrayType(IntegerType, containsNull = false), nullable = true))), | ||
StructField("f1", | ||
ArrayType(StructType(Seq(StructField("num", IntegerType, nullable = false))), | ||
containsNull = false), | ||
nullable = true))), | ||
"""message root { | ||
| optional group f1 (LIST) { | ||
| repeated group element { | ||
| repeated group array { | ||
| required int32 num; | ||
| } | ||
| } | ||
|
@@ -521,7 +525,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | |
StructField("f1", ArrayType(IntegerType, containsNull = false), nullable = true))), | ||
"""message root { | ||
| optional group f1 (LIST) { | ||
| repeated int32 element; | ||
| repeated int32 array; | ||
| } | ||
|} | ||
""".stripMargin) | ||
|
@@ -539,7 +543,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | |
nullable = true))), | ||
"""message root { | ||
| optional group f1 (LIST) { | ||
| repeated group element { | ||
| repeated group array { | ||
| required binary str (UTF8); | ||
| required int32 num; | ||
| } | ||
|
@@ -616,7 +620,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | |
"""message root { | ||
| optional group f1 (LIST) { | ||
| repeated group bag { | ||
| optional int32 element; | ||
| optional int32 array; | ||
| } | ||
| } | ||
|} | ||
|
@@ -648,7 +652,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | |
nullable = true))), | ||
"""message root { | ||
| optional group f1 (LIST) { | ||
| repeated int32 element; | ||
| repeated int32 array; | ||
| } | ||
|} | ||
""".stripMargin) | ||
|
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.
Actually I made a mistake here. We should leave this
array
asarray_element
.This is a little bit complicated... So in the early days, when Spark SQL Parquet support was firstly authored, Parquet format spec wasn't clear about how to write arrays and maps. So Spark SQL took a somewhat weird approach here: if the array may contain nulls, we mimic parquet-hive, which writes a 3-level structure with
array_element
as the 2nd level type name; otherwise, we mimic parquet-avro, which writes a 2-level structure witharray
as the 2nd level type name.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.
Just to be clear, PR #7231 already covers the original bug this PR tried to fix. We'll be able to read Hive data with legacy format. The field names changed here matter for the write path, because we want to write exactly the same format as older Spark SQL versions when compatible mode is turned on.