aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql
diff options
context:
space:
mode:
authorhyukjinkwon <gurwls223@gmail.com>2016-03-21 15:42:35 +0800
committerWenchen Fan <wenchen@databricks.com>2016-03-21 15:42:35 +0800
commite474088144cdd2632cf2fef6b2cf10b3cd191c23 (patch)
tree66374e51a3318ea931ebef343e0a3d285c06d2c5 /python/pyspark/sql
parentf58319a24fd5e026411538b1fb7336d9d894277b (diff)
downloadspark-e474088144cdd2632cf2fef6b2cf10b3cd191c23.tar.gz
spark-e474088144cdd2632cf2fef6b2cf10b3cd191c23.tar.bz2
spark-e474088144cdd2632cf2fef6b2cf10b3cd191c23.zip
[SPARK-13764][SQL] Parse modes in JSON data source
## 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 <gurwls223@gmail.com> Closes #11756 from HyukjinKwon/SPARK-13764.
Diffstat (limited to 'python/pyspark/sql')
-rw-r--r--python/pyspark/sql/readwriter.py8
1 files changed, 8 insertions, 0 deletions
diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py
index 438662bb15..bae9e69df8 100644
--- a/python/pyspark/sql/readwriter.py
+++ b/python/pyspark/sql/readwriter.py
@@ -162,6 +162,14 @@ class DataFrameReader(object):
(e.g. 00012)
* ``allowBackslashEscapingAnyCharacter`` (default ``false``): allows accepting quoting \
of all character using backslash quoting mechanism
+ * ``mode`` (default ``PERMISSIVE``): allows a mode for dealing with corrupt records \
+ during parsing.
+ * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted \
+ 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.
+ * ``FAILFAST`` : throws an exception when it meets corrupted records.
>>> df1 = sqlContext.read.json('python/test_support/sql/people.json')
>>> df1.dtypes