-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-23173][SQL] Avoid creating corrupt parquet files when loading data from JSON #20694
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
|
ok to test |
|
Test build #87776 has finished for PR 20694 at commit
|
| test("from_json missing fields") { | ||
| val conf = SQLConf.get | ||
| for (forceJsonNullableSchema <- Seq(false, true)) { | ||
| conf.setConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA, forceJsonNullableSchema) |
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.
We have to revert the conflicts to the original value.
Thus, move this test to org.apache.spark.sql.execution.datasources.json.JsonSuite. Then, we can use SQLConf
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.
I tried moving the test, but it is not as easy as it may seem. org.apache.spark.sql.execution.datasources.json.JsonSuite lacks checkEvaluation that I'm using here. In general, org.apache.spark.sql.execution.datasources.json.JsonSuite seems to be meant for things such as spark.read.json() and not the from_json() function, so the test would be misplaced there. I think that it's better to keep the test in JsonExpressionsSuite and revert the config.
|
LGTM except one minor comment in the test case. |
|
Test build #88121 has finished for PR 20694 at commit
|
|
retest this please |
|
Test build #88126 has finished for PR 20694 at commit
|
| val schema = JsonToStructs(jsonSchema, Map.empty, Literal.create(input, StringType), gmtId) | ||
| .dataType | ||
| val schemaToCompare = if (forceJsonNullableSchema) jsonSchema.asNullable else jsonSchema | ||
| assert(schemaToCompare == schema); |
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: ; is useless.
| | "c": "foo" | ||
| |} | ||
| |""" | ||
| .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.
Nit: the style.
| | "a": 1, | ||
| | "c": "foo" | ||
| |} | ||
| |""" |
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.
The same here.
|
LGTM Thanks! Merged to master/2.3 I resolved the style issues when I merged the code |
…data from JSON ## What changes were proposed in this pull request? The from_json() function accepts an additional parameter, where the user might specify the schema. The issue is that the specified schema might not be compatible with data. In particular, the JSON data might be missing data for fields declared as non-nullable in the schema. The from_json() function does not verify the data against such errors. When data with missing fields is sent to the parquet encoder, there is no verification either. The end results is a corrupt parquet file. To avoid corruptions, make sure that all fields in the user-specified schema are set to be nullable. Since this changes the behavior of a public function, we need to include it in release notes. The behavior can be reverted by setting `spark.sql.fromJsonForceNullableSchema=false` ## How was this patch tested? Added two new tests. Author: Michał Świtakowski <[email protected]> Closes #20694 from mswit-databricks/SPARK-23173. (cherry picked from commit 2ca9bb0) Signed-off-by: gatorsmile <[email protected]>
…data from JSON ## What changes were proposed in this pull request? The from_json() function accepts an additional parameter, where the user might specify the schema. The issue is that the specified schema might not be compatible with data. In particular, the JSON data might be missing data for fields declared as non-nullable in the schema. The from_json() function does not verify the data against such errors. When data with missing fields is sent to the parquet encoder, there is no verification either. The end results is a corrupt parquet file. To avoid corruptions, make sure that all fields in the user-specified schema are set to be nullable. Since this changes the behavior of a public function, we need to include it in release notes. The behavior can be reverted by setting `spark.sql.fromJsonForceNullableSchema=false` ## How was this patch tested? Added two new tests. Author: Michał Świtakowski <[email protected]> Closes apache#20694 from mswit-databricks/SPARK-23173. (cherry picked from commit 2ca9bb0) Signed-off-by: gatorsmile <[email protected]>
What changes were proposed in this pull request?
The from_json() function accepts an additional parameter, where the user might specify the schema. The issue is that the specified schema might not be compatible with data. In particular, the JSON data might be missing data for fields declared as non-nullable in the schema. The from_json() function does not verify the data against such errors. When data with missing fields is sent to the parquet encoder, there is no verification either. The end results is a corrupt parquet file.
To avoid corruptions, make sure that all fields in the user-specified schema are set to be nullable.
Since this changes the behavior of a public function, we need to include it in release notes.
The behavior can be reverted by setting
spark.sql.fromJsonForceNullableSchema=falseHow was this patch tested?
Added two new tests.