Skip to content

Conversation

@HyukjinKwon
Copy link
Member

What changes were proposed in this pull request?

This PR proposes handling not nullable field properly in JacksonParser. For example, the codes below:

val testJson = """{"nullInt":null}""" :: Nil
val testSchema = StructType(StructField("nullInt", IntegerType, false) :: Nil)
val data = spark.sparkContext.parallelize(testJson)
spark.read.schema(testSchema).json(data).show()

prints

Before

+-------+
|nullInt|
+-------+
|      0|
+-------+

After

org.apache.spark.sql.catalyst.json.NotAllowedNullException: Null not allowed: {"nullInt":null}
    at org.apache.spark.sql.catalyst.json.JacksonParser.failedRecord(JacksonParser.scala:131)
    at org.apache.spark.sql.catalyst.json.JacksonParser.parse(JacksonParser.scala:472)
...

How was this patch tested?

Unit test inJsonSuite.

@SparkQA
Copy link

SparkQA commented Oct 3, 2016

Test build #66257 has finished for PR 15329 at commit 8824186.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member Author

retest this please

@HyukjinKwon
Copy link
Member Author

The tests in JDBCWriteSuite are locally passed, so I re-ran this.

@SparkQA
Copy link

SparkQA commented Oct 3, 2016

Test build #66258 has finished for PR 15329 at commit 8824186.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 3, 2016

Test build #66260 has finished for PR 15329 at commit cb5ece7.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member Author

Hi @yhuai and @cloud-fan , I recall changing codes here was reviewed by you both. Do you mind if I ask to review this please?


private[sql] class SparkSQLJsonProcessingException(msg: String) extends RuntimeException(msg)

private[sql] class NotAllowedNullException(msg: String) extends SparkSQLJsonProcessingException(msg)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need this exception?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added this one to differentiate this case from other SparkSQLJsonProcessingException[1] because when it fails to parse due to not-allowed null, permissive parse mode is not allowed (because permissive mode virtually means allowing nulls on other fields).

[1]https://github.com/HyukjinKwon/spark/blob/cb5ece70bc840820697fa73943bc28bd63c96b41/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala#L471-L474

}
Nil
} else {
throw new NotAllowedNullException(s"Null not allowed: $record")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please explain why null is not allowed in the error message. Is it possible to know which field is not nullable? I am not sure if just printing the record is really helpful at here (considering users may often have long JSON records).

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, let me try to produce a better message.

@yhuai
Copy link
Contributor

yhuai commented Oct 8, 2016

Actually, when will a user want to specify non-nullable for any json field? I am not sure if we are actually addressing the right problem. I am wondering if we should just not allow non-nullable fields for json.

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Oct 8, 2016

@yhuai Sure, I support that idea too as I think in the same way. These would be minor cases. However, just the case I thought instantly is, to use the same schema used as it is (because inferring schema in JSON is discouraged in production). For example,

df.write.format("json").save(...)
...
spark.read.schema(df.schema).load(...)

or

anotherDF.select(from_json("a", df.schema))

In that case, I guess the original df could contain non-nullable fields.

FYI, as you might already know, it forces to the nullable schema reading/writing it from files but not for structured streaming/reading from rdd/from_json. I opened a PR for the consistency before (#14124).

@HyukjinKwon
Copy link
Member Author

I am willing to follow your decision if you confirm.

@yhuai
Copy link
Contributor

yhuai commented Oct 8, 2016

Yea. That's a good point. If we do not allow non-nullable fields, we should also let users easily convert nullability field. Let me also check with @marmbrus.

@HyukjinKwon
Copy link
Member Author

After re-thinking, I think this might be blocked by #14124 (or at least closely related). Anyhow, it looks dependent of it. Let me close this for now and try to reopen this if I can feel able to proceed this further independently.

@HyukjinKwon HyukjinKwon closed this Jan 5, 2017
@HyukjinKwon HyukjinKwon deleted the SPARK-17763 branch January 2, 2018 03:39
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants