Skip to content

Commit

Permalink
[SPARK-16548][SQL] Inconsistent error handling in JSON parsing SQL fu…
Browse files Browse the repository at this point in the history
…nctions

## What changes were proposed in this pull request?

change to using Jackson's `com.fasterxml.jackson.core.JsonFactory`

    public JsonParser createParser(String content)

## How was this patch tested?

existing unit tests

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Eric Wasserman <ericw@sgn.com>

Closes #17693 from ewasserman/SPARK-20314.

(cherry picked from commit 57e1da3)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
Eric Wasserman authored and cloud-fan committed Apr 26, 2017
1 parent f0de600 commit c8803c0
Show file tree
Hide file tree
Showing 2 changed files with 26 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.sql.catalyst.expressions

import java.io.{ByteArrayOutputStream, CharArrayWriter, StringWriter}
import java.io.{ByteArrayInputStream, ByteArrayOutputStream, CharArrayWriter, InputStreamReader, StringWriter}

import scala.util.parsing.combinator.RegexParsers

Expand Down Expand Up @@ -149,7 +149,10 @@ case class GetJsonObject(json: Expression, path: Expression)

if (parsed.isDefined) {
try {
Utils.tryWithResource(jsonFactory.createParser(jsonStr.getBytes)) { parser =>
/* We know the bytes are UTF-8 encoded. Pass a Reader to avoid having Jackson
detect character encoding which could fail for some malformed strings */
Utils.tryWithResource(jsonFactory.createParser(new InputStreamReader(
new ByteArrayInputStream(jsonStr.getBytes), "UTF-8"))) { parser =>
val output = new ByteArrayOutputStream()
val matched = Utils.tryWithResource(
jsonFactory.createGenerator(output, JsonEncoding.UTF8)) { generator =>
Expand Down Expand Up @@ -393,7 +396,10 @@ case class JsonTuple(children: Seq[Expression])
}

try {
Utils.tryWithResource(jsonFactory.createParser(json.getBytes)) {
/* We know the bytes are UTF-8 encoded. Pass a Reader to avoid having Jackson
detect character encoding which could fail for some malformed strings */
Utils.tryWithResource(jsonFactory.createParser(new InputStreamReader(
new ByteArrayInputStream(json.getBytes), "UTF-8"))) {
parser => parseRow(parser, input)
}
} catch {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,10 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
|"fb:testid":"1234"}
|""".stripMargin

/* invalid json with leading nulls would trigger java.io.CharConversionException
in Jackson's JsonFactory.createParser(byte[]) due to RFC-4627 encoding detection */
val badJson = "\0\0\0A\1AAA"

test("$.store.bicycle") {
checkEvaluation(
GetJsonObject(Literal(json), Literal("$.store.bicycle")),
Expand Down Expand Up @@ -224,6 +228,13 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
null)
}

test("SPARK-16548: character conversion") {
checkEvaluation(
GetJsonObject(Literal(badJson), Literal("$.a")),
null
)
}

test("non foldable literal") {
checkEvaluation(
GetJsonObject(NonFoldableLiteral(json), NonFoldableLiteral("$.fb:testid")),
Expand Down Expand Up @@ -340,6 +351,12 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
InternalRow(null, null, null, null, null))
}

test("SPARK-16548: json_tuple - invalid json with leading nulls") {
checkJsonTuple(
JsonTuple(Literal(badJson) :: jsonTupleQuery),
InternalRow(null, null, null, null, null))
}

test("json_tuple - preserve newlines") {
checkJsonTuple(
JsonTuple(Literal("{\"a\":\"b\nc\"}") :: Literal("a") :: Nil),
Expand Down

0 comments on commit c8803c0

Please sign in to comment.