Skip to content
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-13764][SQL] Parse modes in JSON data source #11756

Closed
wants to merge 13 commits into from

Conversation

HyukjinKwon
Copy link
Member

What changes were proposed in this pull request?

Currently, there is no way to control the behaviour when fails to parse corrupt records in JSON data source .

This PR adds the support for parse modes just like CSV data source. There are three modes below:

  • PERMISSIVE : When it fails to parse, this sets null to to field. This is a default mode when it has been this mode.
  • DROPMALFORMED: When it fails to parse, this drops the whole record.
  • FAILFAST: When it fails to parse, it just throws an exception.

This PR also make JSON data source share the ParseModes in CSV data source.

How was this patch tested?

Unit tests were used and ./dev/run_tests for code style tests.

@HyukjinKwon HyukjinKwon changed the title Parse modes in JSON data source [SPARK-13764][SQL] Parse modes in JSON data source Mar 16, 2016
@rxin
Copy link
Contributor

rxin commented Mar 16, 2016

cc @cloud-fan for review

def isPermissiveMode(mode: String): Boolean = if (isValidMode(mode)) {
mode.toUpperCase == PERMISSIVE_MODE
} else {
true // We default to permissive is the mode string is not valid
Copy link
Contributor

Choose a reason for hiding this comment

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

should we log a warning for this case?

@SparkQA
Copy link

SparkQA commented Mar 16, 2016

Test build #53286 has finished for PR 11756 at commit 4c46f4b.

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

@@ -288,6 +288,9 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging {
* </li>
* <li>`allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers
* (e.g. 00012)</li>
* <li>`mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records
* during parsing. When fails to parse, `PERMISSIVE` mode sets `null`, `DROPMALFORMED` drops the
* record and `FAILFAST` throws an exception.<li>
Copy link
Contributor

Choose a reason for hiding this comment

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

I think we need to say more about these 3 modes. From the tests, it looks to me that:

  • PERMISSIVE mode will set other fields to null when meet a corrupted record, and put the malformed string into a new field configured by spark.sql.columnNameOfCorruptRecord.
  • DROPMALFORMED mode will ignore corrupted records and append a new field which is always null to the output.
  • FAILFAST mode will throw an exception.

It will be better if you can expand this doc and add some examples.

Copy link
Member Author

Choose a reason for hiding this comment

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

Could I maybe edit this without some examples? It is becoming a bit messy..

@cloud-fan
Copy link
Contributor

I'm not familiar with CSV part, what if users set the schema directly before read data and the mode is PERMISSIVE? Will we add the extra field?

@HyukjinKwon
Copy link
Member Author

For example, the data below:

1,2,3,4
3,2,1

will produce the records below:

  • PERMISSIVE
Row(1,2,3,4)
Row(3,2,1,null)
  • PERMISSIVE with user schema
Schema("field1", "field2", "field3")
Row(1,2,3)
Row(3,2,1)
  • PERMISSIVE with user schema
Schema("field1", "field2", "field3", "field4", "field5")
Row(1,2,3,4,null)
Row(3,2,1,null,null)
  • DROPMALFORMED
Row(1,2,3,4)
  • DROPMALFORMED with user schema
Schema("field1", "field2", "field3")
Drop all.
  • DROPMALFORMED with user schema
Schema("field1", "field2", "field3", "field4", "field5")
Drop all.
  • FAILFAST
Throws an exception
  • FAILFAST with user schema
Schema("field1", "field2", "field3")
Throws an exception.
  • FAILFAST with user schema
Schema("field1", "field2", "field3", "field4", "field5")
Throws an exception.

@cloud-fan I just added some more cases here.

@SparkQA
Copy link

SparkQA commented Mar 16, 2016

Test build #53303 has finished for PR 11756 at commit 3675fae.

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

@HyukjinKwon
Copy link
Member Author

The commit I submitted includes comment changes and avoiding to add a _corrupt_record field when it is DROPMALFORMED mode in type inference.

@SparkQA
Copy link

SparkQA commented Mar 16, 2016

Test build #53313 has finished for PR 11756 at commit 32ae8b2.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Mar 16, 2016

Test build #53312 has finished for PR 11756 at commit 4440a55.

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

@cloud-fan
Copy link
Contributor

ah, thanks for the detail explanation and examples!

record and puts the malformed string into a new field configured by \
``spark.sql.columnNameOfCorruptRecord``. When a schema is set by user, it sets \
``null`` for extra fields.
* ``DROPMALFORMED`` : ignores the whole corrupted records and append.
Copy link
Contributor

Choose a reason for hiding this comment

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

should remove and append?

@cloud-fan
Copy link
Contributor

Overall LGTM, thanks for working on it!

@HyukjinKwon
Copy link
Member Author

@cloud-fan Actually, I have a question.
So, in JSON data source, I thought JSON data format itself can have a flexible schema so it does not necessarily have the same data unlike CSV data.

So, I thought the range of "malformed" rows does not include some rows having different schema for JSON data source (whereas for CSV the range of "malformed" rows includes some rows having different schema).

For the differences, it lead to some different actions for each parse mode comparing to CSV data source.

  • CSV
    • FAILFAST : It throws an exception if any row does not have a same schema or if any row could not be converted into the user-given schema.
    • DROPMALFORMED : : It drops every row that does not have a same schema or could not be converted into the user-given schema.
  • JSON
    • FAILFAST : It throws an exception if any row has a corrupted format or if any row could not be converted into the user-given schema.
    • DROPMALFORMED : It drops every row that has a corrupted format or could not be converted into the user-given schema.

Do you think it is acceptable?

@cloud-fan
Copy link
Contributor

This makes sense to me. Actually for CSV, when any row does not have a same schema, it just means corrupted format, as CSV has a very simple format and can always be parsed.

@cloud-fan
Copy link
Contributor

LGTM, cc @davies for another look.

@rxin
Copy link
Contributor

rxin commented Mar 17, 2016

But since we had it, i'd say we should keep it to avoid breaking compatibility. We can have the per-read option override the global option.

@SparkQA
Copy link

SparkQA commented Mar 17, 2016

Test build #53379 has finished for PR 11756 at commit de8d291.

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

@HyukjinKwon
Copy link
Member Author

Filed in SPARK-13953.

@SparkQA
Copy link

SparkQA commented Mar 17, 2016

Test build #53382 has finished for PR 11756 at commit 29a8f68.

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

@SparkQA
Copy link

SparkQA commented Mar 17, 2016

Test build #53384 has finished for PR 11756 at commit 551593a.

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

@SparkQA
Copy link

SparkQA commented Mar 18, 2016

Test build #53504 has finished for PR 11756 at commit bfc0405.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Mar 18, 2016

Test build #53506 has finished for PR 11756 at commit 59e7214.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Mar 18, 2016

Test build #53507 has finished for PR 11756 at commit 3ff900e.

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

@cloud-fan
Copy link
Contributor

retest it please.

@HyukjinKwon
Copy link
Member Author

@cloud-fan Isn't this "it" a typo maybe :)?

@HyukjinKwon
Copy link
Member Author

retest this please

Row("str_a_4") :: Nil)
assert(jsonDFTwo.schema === schemaTwo)
}

test("Corrupt records") {
Copy link
Contributor

Choose a reason for hiding this comment

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

we can change this to: "Corrupt records: PERMISSIVE mode"

@SparkQA
Copy link

SparkQA commented Mar 21, 2016

Test build #53652 has finished for PR 11756 at commit 3ff900e.

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

@cloud-fan
Copy link
Contributor

Last 2 comments, otherwise LGTM

@cloud-fan
Copy link
Contributor

LGTM, pending tests

@SparkQA
Copy link

SparkQA commented Mar 21, 2016

Test build #53659 has finished for PR 11756 at commit dec3d81.

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

@cloud-fan
Copy link
Contributor

Thanks! Merging to master!

@asfgit asfgit closed this in e474088 Mar 21, 2016
roygao94 pushed a commit to roygao94/spark that referenced this pull request Mar 22, 2016
## What changes were proposed in this pull request?

Currently, there is no way to control the behaviour when fails to parse corrupt records in JSON data source .

This PR adds the support for parse modes just like CSV data source. There are three modes below:

- `PERMISSIVE` :  When it fails to parse, this sets `null` to to field. This is a default mode when it has been this mode.
- `DROPMALFORMED`: When it fails to parse, this drops the whole record.
- `FAILFAST`: When it fails to parse, it just throws an exception.

This PR also make JSON data source share the `ParseModes` in CSV data source.

## How was this patch tested?

Unit tests were used and `./dev/run_tests` for code style tests.

Author: hyukjinkwon <[email protected]>

Closes apache#11756 from HyukjinKwon/SPARK-13764.
HyukjinKwon added a commit to databricks/spark-xml that referenced this pull request Sep 10, 2016
#105

Currently, this library does not support `PERMISSIVE` parse mode. Similar with JSON data source, this also can be done in the same way with `_corrupt_record`.

This PR adds the support for `PERMISSIVE` mode and make this behaviour consistent with the other data sources supporting parse modes (JSON and CSV data sources.)

Also, this PR adds the support for `_corrupt_record`.

This PR is similar with apache/spark#11756 and apache/spark#11881.

Author: hyukjinkwon <[email protected]>

Closes #107 from HyukjinKwon/ISSUE-105-permissive.
@HyukjinKwon HyukjinKwon deleted the SPARK-13764 branch January 2, 2018 03:42
beluisterql added a commit to beluisterql/spark-xml that referenced this pull request Aug 4, 2024
databricks/spark-xml#105

Currently, this library does not support `PERMISSIVE` parse mode. Similar with JSON data source, this also can be done in the same way with `_corrupt_record`.

This PR adds the support for `PERMISSIVE` mode and make this behaviour consistent with the other data sources supporting parse modes (JSON and CSV data sources.)

Also, this PR adds the support for `_corrupt_record`.

This PR is similar with apache/spark#11756 and apache/spark#11881.

Author: hyukjinkwon <[email protected]>

Closes #107 from HyukjinKwon/ISSUE-105-permissive.
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