diff options
author | hyukjinkwon <gurwls223@gmail.com> | 2017-03-05 14:35:06 -0800 |
---|---|---|
committer | Burak Yavuz <brkyvz@gmail.com> | 2017-03-05 14:35:06 -0800 |
commit | 369a148e591bb16ec7da54867610b207602cd698 (patch) | |
tree | e7c2469ce548557bef43d3ccdb0fee6d5c006ec5 /python/pyspark | |
parent | 80d5338b32e856870cf187ce17bc87335d690761 (diff) | |
download | spark-369a148e591bb16ec7da54867610b207602cd698.tar.gz spark-369a148e591bb16ec7da54867610b207602cd698.tar.bz2 spark-369a148e591bb16ec7da54867610b207602cd698.zip |
[SPARK-19595][SQL] Support json array in from_json
## What changes were proposed in this pull request?
This PR proposes to both,
**Do not allow json arrays with multiple elements and return null in `from_json` with `StructType` as the schema.**
Currently, it only reads the single row when the input is a json array. So, the codes below:
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = StructType(StructField("a", IntegerType) :: Nil)
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("struct").select(from_json(col("struct"), schema)).show()
```
prints
```
+--------------------+
|jsontostruct(struct)|
+--------------------+
| [1]|
+--------------------+
```
This PR simply suggests to print this as `null` if the schema is `StructType` and input is json array.with multiple elements
```
+--------------------+
|jsontostruct(struct)|
+--------------------+
| null|
+--------------------+
```
**Support json arrays in `from_json` with `ArrayType` as the schema.**
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("array").select(from_json(col("array"), schema)).show()
```
prints
```
+-------------------+
|jsontostruct(array)|
+-------------------+
| [[1], [2]]|
+-------------------+
```
## How was this patch tested?
Unit test in `JsonExpressionsSuite`, `JsonFunctionsSuite`, Python doctests and manual test.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #16929 from HyukjinKwon/disallow-array.
Diffstat (limited to 'python/pyspark')
-rw-r--r-- | python/pyspark/sql/functions.py | 11 |
1 files changed, 8 insertions, 3 deletions
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 426a4a8c93..376b86ea69 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1773,11 +1773,11 @@ def json_tuple(col, *fields): @since(2.1) def from_json(col, schema, options={}): """ - Parses a column containing a JSON string into a [[StructType]] with the - specified schema. Returns `null`, in the case of an unparseable string. + Parses a column containing a JSON string into a [[StructType]] or [[ArrayType]] + with the specified schema. Returns `null`, in the case of an unparseable string. :param col: string column in json format - :param schema: a StructType to use when parsing the json column + :param schema: a StructType or ArrayType to use when parsing the json column :param options: options to control parsing. accepts the same options as the json datasource >>> from pyspark.sql.types import * @@ -1786,6 +1786,11 @@ def from_json(col, schema, options={}): >>> df = spark.createDataFrame(data, ("key", "value")) >>> df.select(from_json(df.value, schema).alias("json")).collect() [Row(json=Row(a=1))] + >>> data = [(1, '''[{"a": 1}]''')] + >>> schema = ArrayType(StructType([StructField("a", IntegerType())])) + >>> df = spark.createDataFrame(data, ("key", "value")) + >>> df.select(from_json(df.value, schema).alias("json")).collect() + [Row(json=[Row(a=1)])] """ sc = SparkContext._active_spark_context |