-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-35912][SQL] Fix nullability of spark.read.json
#33212
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
6f29bea
1b8dade
b1d749f
48b8006
e44348d
3cfc583
51923c5
3f3339c
4f2e904
0f5354b
9b08de1
484a17e
7a6d978
9fb8fbd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -405,10 +405,18 @@ class JacksonParser( | |
| schema.getFieldIndex(parser.getCurrentName) match { | ||
| case Some(index) => | ||
| try { | ||
| row.update(index, fieldConverters(index).apply(parser)) | ||
| val fieldValue = fieldConverters(index).apply(parser) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I .. am not sure if we really need these complicated fix to address nullability mismatch (which is rather a corner case) to be honest. I wonder if there's a simpler approach, e.g.) simply warning on non-nullable columns?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's another proposal I mentioned earlier: if the user-given schema is not-nullable, we just turn it into nullable schema and don't fail.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thank you for your suggestions, I'll raise a new PR. |
||
| val isIllegal = | ||
| options.parseMode != PermissiveMode && !schema(index).nullable && fieldValue == null | ||
| if (isIllegal) { | ||
| throw new IllegalSchemaArgumentException( | ||
| s"field ${schema(index).name} is not nullable but the parsed value is null.") | ||
| } | ||
| row.update(index, fieldValue) | ||
| skipRow = structFilters.skipRow(row, index) | ||
| } catch { | ||
| case e: SparkUpgradeException => throw e | ||
| case e: IllegalSchemaArgumentException => throw e | ||
| case NonFatal(e) if isRoot => | ||
| badRecordException = badRecordException.orElse(Some(e)) | ||
| parser.skipChildren() | ||
|
|
@@ -418,6 +426,9 @@ class JacksonParser( | |
| } | ||
| } | ||
|
|
||
| // When the input schema is setting to `nullable = false`, make sure the field is not null. | ||
cfmcgrady marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| checkNotNullableInRow(row, schema, skipRow, badRecordException) | ||
|
|
||
| if (skipRow) { | ||
| None | ||
| } else if (badRecordException.isEmpty) { | ||
|
|
@@ -427,6 +438,28 @@ class JacksonParser( | |
| } | ||
| } | ||
|
|
||
| // As PERMISSIVE mode only works with nullable fields, we can skip this not nullable check when | ||
| // the mode is PERMISSIVE. (see FailureSafeParser.checkNullabilityForPermissiveMode) | ||
| private lazy val checkNotNullableInRow = if (options.parseMode != PermissiveMode) { | ||
| (row: GenericInternalRow, | ||
| schema: StructType, | ||
| skipRow: Boolean, | ||
| runtimeExceptionOption: Option[Throwable]) => { | ||
| if (runtimeExceptionOption.isEmpty && !skipRow) { | ||
| var index = 0 | ||
| while (index < schema.length) { | ||
| if (!schema(index).nullable && row.isNullAt(index)) { | ||
| throw new IllegalSchemaArgumentException( | ||
| s"field ${schema(index).name} is not nullable but it's missing in one record.") | ||
| } | ||
| index += 1 | ||
| } | ||
| } | ||
| } | ||
| } else { | ||
| (_: GenericInternalRow, _: StructType, _: Boolean, _: Option[Throwable]) => {} | ||
| } | ||
|
|
||
| /** | ||
| * Parse an object as a Map, preserving all fields. | ||
| */ | ||
|
|
@@ -483,6 +516,7 @@ class JacksonParser( | |
| } | ||
| } catch { | ||
| case e: SparkUpgradeException => throw e | ||
| case e: IllegalSchemaArgumentException => throw e | ||
| case e @ (_: RuntimeException | _: JsonProcessingException | _: MalformedInputException) => | ||
| // JSON parser currently doesn't support partial results for corrupted records. | ||
| // For such records, all fields other than the field configured by | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -924,7 +924,7 @@ abstract class JsonSuite | |
| test("Applying schemas with MapType") { | ||
| withTempView("jsonWithSimpleMap", "jsonWithComplexMap") { | ||
| val schemaWithSimpleMap = StructType( | ||
| StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) | ||
| StructField("map", MapType(StringType, IntegerType, true), true) :: Nil) | ||
| val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1) | ||
|
|
||
| jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap") | ||
|
|
@@ -953,7 +953,7 @@ abstract class JsonSuite | |
| StructField("field1", ArrayType(IntegerType, true), true) :: | ||
| StructField("field2", IntegerType, true) :: Nil) | ||
| val schemaWithComplexMap = StructType( | ||
| StructField("map", MapType(StringType, innerStruct, true), false) :: Nil) | ||
| StructField("map", MapType(StringType, innerStruct, true), true) :: Nil) | ||
|
|
||
| val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2) | ||
|
|
||
|
|
@@ -1392,7 +1392,7 @@ abstract class JsonSuite | |
| withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { | ||
| withTempDir { dir => | ||
| val schemaWithSimpleMap = StructType( | ||
| StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) | ||
| StructField("map", MapType(StringType, IntegerType, true), true) :: Nil) | ||
| val df = spark.read.schema(schemaWithSimpleMap).json(mapType1) | ||
|
|
||
| val path = dir.getAbsolutePath | ||
|
|
@@ -2919,6 +2919,55 @@ abstract class JsonSuite | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-35912: nullability with different parse mode -- struct") { | ||
| // JSON field is missing. | ||
| val missingFieldInput = """{"c1": 1}""" | ||
| // JSON filed is null. | ||
| val nullValueInput = """{"c1": 1, "c2": null}""" | ||
|
|
||
| val load = (mode: String, schema: StructType, inputJson: String) => { | ||
| val json = spark.createDataset( | ||
| spark.sparkContext.parallelize(inputJson :: Nil))(Encoders.STRING) | ||
| spark.read | ||
| .option("mode", mode) | ||
| .schema(schema) | ||
| .json(json) | ||
| } | ||
|
|
||
| Seq(true, false).foreach { nullable => | ||
cfmcgrady marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| val schema = StructType(Seq( | ||
| StructField("c1", IntegerType, nullable = true), | ||
| StructField("c2", IntegerType, nullable = nullable))) | ||
|
|
||
| Seq(missingFieldInput, nullValueInput).foreach { jsonString => | ||
| if (nullable) { | ||
| checkAnswer(load("DROPMALFORMED", schema, jsonString), Row(1, null) :: Nil) | ||
| checkAnswer(load("FAILFAST", schema, jsonString), Row(1, null) :: Nil) | ||
| checkAnswer(load("PERMISSIVE", schema, jsonString), Row(1, null) :: Nil) | ||
| } else { | ||
| checkAnswer(load("DROPMALFORMED", schema, jsonString), Seq.empty) | ||
|
|
||
| val exceptionMsg1 = intercept[SparkException] { | ||
| load("FAILFAST", schema, jsonString).collect | ||
| }.getMessage | ||
| val expectedMsg1 = if (jsonString == missingFieldInput) { | ||
| "field c2 is not nullable but it's missing in one record." | ||
| } else { | ||
| s"field c2 is not nullable but the parsed value is null." | ||
| } | ||
| assert(exceptionMsg1.contains(expectedMsg1)) | ||
|
|
||
| val exceptionMsg2 = intercept[SparkException] { | ||
| load("PERMISSIVE", schema, jsonString).collect | ||
|
||
| } | ||
| val expectedMsg2 = | ||
| "Field c2 is not nullable but PERMISSIVE mode only works with nullable fields." | ||
|
||
| assert(exceptionMsg2.getMessage.contains(expectedMsg2)) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| class JsonV1Suite extends JsonSuite { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This change also affects parsing CSV string, I will add a unit test for CSV in a separate pr. cc @cloud-fan @HyukjinKwon