[SPARK-20775][SQL] Added scala support from_json
## What changes were proposed in this pull request? from_json function required to take in a java.util.Hashmap. For other functions, a java wrapper is provided which casts a java hashmap to a scala map. Only a java function is provided in this case, forcing scala users to pass in a java.util.Hashmap. Added the missing wrapper. ## How was this patch tested? Added a unit test for passing in a scala map Author: setjet <rubenljanssen@gmail.com> Closes #18094 from setjet/spark-20775.
This commit is contained in:
parent
c1e7989c4f
commit
2dbe0c5288
|
@ -3060,8 +3060,9 @@ object functions {
|
||||||
from_json(e, schema, Map.empty[String, String])
|
from_json(e, schema, Map.empty[String, String])
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Parses a column containing a JSON string into a `StructType` or `ArrayType` of `StructType`s
|
* (Java-specific) 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.
|
* of `StructType`s with the specified schema. Returns `null`, in the case of an unparseable
|
||||||
|
* string.
|
||||||
*
|
*
|
||||||
* @param e a string column containing JSON data.
|
* @param e a string column containing JSON data.
|
||||||
* @param schema the schema to use when parsing the json string as a json string. In Spark 2.1,
|
* @param schema the schema to use when parsing the json string as a json string. In Spark 2.1,
|
||||||
|
@ -3072,6 +3073,23 @@ object functions {
|
||||||
* @since 2.1.0
|
* @since 2.1.0
|
||||||
*/
|
*/
|
||||||
def from_json(e: Column, schema: String, options: java.util.Map[String, String]): Column = {
|
def from_json(e: Column, schema: String, options: java.util.Map[String, String]): Column = {
|
||||||
|
from_json(e, schema, options.asScala.toMap)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* (Scala-specific) Parses a column containing a JSON string into a `StructType` or `ArrayType`
|
||||||
|
* of `StructType`s with the specified schema. Returns `null`, in the case of an unparseable
|
||||||
|
* string.
|
||||||
|
*
|
||||||
|
* @param e a string column containing JSON data.
|
||||||
|
* @param schema the schema to use when parsing the json string as a json string. In Spark 2.1,
|
||||||
|
* the user-provided schema has to be in JSON format. Since Spark 2.2, the DDL
|
||||||
|
* format is also supported for the schema.
|
||||||
|
*
|
||||||
|
* @group collection_funcs
|
||||||
|
* @since 2.3.0
|
||||||
|
*/
|
||||||
|
def from_json(e: Column, schema: String, options: Map[String, String]): Column = {
|
||||||
val dataType = try {
|
val dataType = try {
|
||||||
DataType.fromJson(schema)
|
DataType.fromJson(schema)
|
||||||
} catch {
|
} catch {
|
||||||
|
|
|
@ -156,13 +156,20 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||||
Row(Seq(Row(1, "a"), Row(2, null), Row(null, null))))
|
Row(Seq(Row(1, "a"), Row(2, null), Row(null, null))))
|
||||||
}
|
}
|
||||||
|
|
||||||
test("from_json uses DDL strings for defining a schema") {
|
test("from_json uses DDL strings for defining a schema - java") {
|
||||||
val df = Seq("""{"a": 1, "b": "haa"}""").toDS()
|
val df = Seq("""{"a": 1, "b": "haa"}""").toDS()
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
df.select(from_json($"value", "a INT, b STRING", new java.util.HashMap[String, String]())),
|
df.select(from_json($"value", "a INT, b STRING", new java.util.HashMap[String, String]())),
|
||||||
Row(Row(1, "haa")) :: Nil)
|
Row(Row(1, "haa")) :: Nil)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
test("from_json uses DDL strings for defining a schema - scala") {
|
||||||
|
val df = Seq("""{"a": 1, "b": "haa"}""").toDS()
|
||||||
|
checkAnswer(
|
||||||
|
df.select(from_json($"value", "a INT, b STRING", Map[String, String]())),
|
||||||
|
Row(Row(1, "haa")) :: Nil)
|
||||||
|
}
|
||||||
|
|
||||||
test("to_json - struct") {
|
test("to_json - struct") {
|
||||||
val df = Seq(Tuple1(Tuple1(1))).toDF("a")
|
val df = Seq(Tuple1(Tuple1(1))).toDF("a")
|
||||||
|
|
||||||
|
|
Loading…
Reference in a new issue