Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ class JacksonParser(
new NoopFilters
}
(parser: JsonParser) => parseJsonToken[Iterable[InternalRow]](parser, st) {
case START_OBJECT => convertObject(parser, st, fieldConverters, jsonFilters)
case START_OBJECT => convertObject(parser, st, fieldConverters, jsonFilters, isRoot = true)
// SPARK-3308: support reading top level JSON arrays and take every element
// in such an array as a row
//
Expand Down Expand Up @@ -383,7 +383,8 @@ class JacksonParser(
parser: JsonParser,
schema: StructType,
fieldConverters: Array[ValueConverter],
structFilters: StructFilters = new NoopFilters()): Option[InternalRow] = {
structFilters: StructFilters = new NoopFilters(),
isRoot: Boolean = false): Option[InternalRow] = {
val row = new GenericInternalRow(schema.length)
var badRecordException: Option[Throwable] = None
var skipRow = false
Expand All @@ -397,7 +398,7 @@ class JacksonParser(
skipRow = structFilters.skipRow(row, index)
} catch {
case e: SparkUpgradeException => throw e
case NonFatal(e) =>
case NonFatal(e) if isRoot =>
badRecordException = badRecordException.orElse(Some(e))
parser.skipChildren()
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -733,4 +733,18 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession {
| """.stripMargin)
checkAnswer(toDF("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), toDF("yyyy-MM-dd'T'HH:mm:ss[.SSS][XXX]"))
}

test("SPARK-33134: return partial results only for root JSON objects") {
val st = new StructType()
.add("c1", LongType)
.add("c2", ArrayType(new StructType().add("c3", LongType).add("c4", StringType)))
val df1 = Seq("""{"c2": [19], "c1": 123456}""").toDF("c0")
checkAnswer(df1.select(from_json($"c0", st)), Row(Row(123456, null)))
val df2 = Seq("""{"data": {"c2": [19], "c1": 123456}}""").toDF("c0")
checkAnswer(df2.select(from_json($"c0", new StructType().add("data", st))), Row(Row(null)))
val df3 = Seq("""[{"c2": [19], "c1": 123456}]""").toDF("c0")
checkAnswer(df3.select(from_json($"c0", ArrayType(st))), Row(null))
val df4 = Seq("""{"c2": [19]}""").toDF("c0")
checkAnswer(df4.select(from_json($"c0", MapType(StringType, st))), Row(null))
}
}