-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-29444] Add configuration to support JacksonGenrator to keep fields with null values #26098
Conversation
Previous discussion about this: SPARK-23773 |
cc @cloud-fan |
Hi, @stczwd . Thank you for making a PR. |
@@ -39,6 +39,18 @@ class JacksonGeneratorSuite extends SparkFunSuite { | |||
assert(writer.toString === """{"a":1}""") | |||
} | |||
|
|||
test("initial with StructType and write out an empty row with allowStructIncludeNull=true") { |
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 looks like a test case for bug. We need a JIRA issue ID prefix for the test case name.
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.
ok
Yes, this PR can resolve this problem |
cc @dongjoon-hyun @xuanyuanking @cloud-fan @rdblue |
Sounds fine to me. |
@@ -76,6 +76,9 @@ private[sql] class JSONOptions( | |||
// Whether to ignore column of all null values or empty array/struct during schema inference | |||
val dropFieldIfAllNull = parameters.get("dropFieldIfAllNull").map(_.toBoolean).getOrElse(false) | |||
|
|||
// Whether to ignore column of all null during json generating | |||
val structIngoreNull = parameters.getOrElse("structIngoreNull", "true").toBoolean |
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.
is it specific to struct type column? if not how about naming it ignroeNullFields
?
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.
It works on StructType, including struct field and struct inner data
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.
how about top-level columns?
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.
Yeap, it also works on that
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.
then shall we pick a better name for this config?
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.
okey, ignoreNullFields is much better than structIgnoreNull, I'll change it.
The change looks reasonable. Do you know why the json data source ignore null fields at the first place? |
val gen = new JacksonGenerator(dataType, writer, allowNullOption) | ||
gen.write(input) | ||
gen.flush() | ||
assert(writer.toString === """{"a":null}""") |
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.
can we also test null inner field? e.g. {"a": {"b": null}}
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.
Sure, I have added a test for this
@cloud-fan I don't know about this. Any ideas? |
@cloud-fan @dongjoon-hyun |
@stczwd I have a question. After this change when we write to the json file, we will be able to preserve null values. Right ?
|
@dilipbiswal Yes, but case is not right. If you want to preserve null values, you must write like this to disable ignoreNullFields.
|
@stczwd Thanks.. sorry .. i missed the newly added option. |
@@ -1153,6 +1153,12 @@ object SQLConf { | |||
.booleanConf | |||
.createWithDefault(true) | |||
|
|||
val JSON_GENERATOR_IGNORE_NULL_FIELDS = | |||
buildConf("spark.sql.jsonGenerator.nullFields.ignore") |
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.
config name has namespaces, we'd better avoid creating unnecessary namespaces. How about spark.sql.jsonGenerator.ignoreNullFields
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.
okey, I will change it.
@@ -76,6 +76,9 @@ private[sql] class JSONOptions( | |||
// Whether to ignore column of all null values or empty array/struct during schema inference | |||
val dropFieldIfAllNull = parameters.get("dropFieldIfAllNull").map(_.toBoolean).getOrElse(false) | |||
|
|||
// Whether to ignore null fields during json generating | |||
val ignoreNullFields = parameters.getOrElse("ignoreNullFields", "true").toBoolean |
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.
SQLConf should also take effect here. How about
parameters.get("ignoreNullFields").map(_.toBoolean).getOrElse {
SQLConf.get. jsonGeneratorIgnoreNullFields
}
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.
Perfect. Then the code in DataSet.scala is not necessary now.
@cloud-fan Any more questions? |
ok to test |
The code changes LGTM. |
Test build #112218 has finished for PR 26098 at commit
|
@@ -76,6 +77,10 @@ private[sql] class JSONOptions( | |||
// Whether to ignore column of all null values or empty array/struct during schema inference | |||
val dropFieldIfAllNull = parameters.get("dropFieldIfAllNull").map(_.toBoolean).getOrElse(false) | |||
|
|||
// Whether to ignore null fields during json generating | |||
val ignoreNullFields = parameters.getOrElse("ignoreNullFields", |
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.
Hey, you should document this in DataFrameWrtier
, DataStreamWrtier
, readwriter.py
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.
Okey, I'll try to follow up this PR, and add this to document
@@ -1153,6 +1153,12 @@ object SQLConf { | |||
.booleanConf | |||
.createWithDefault(true) | |||
|
|||
val JSON_GENERATOR_IGNORE_NULL_FIELDS = | |||
buildConf("spark.sql.jsonGenerator.ignoreNullFields") | |||
.doc("If false, JacksonGenerator will generate null for null fields in Struct.") |
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.
If I were a user, I would have no idea what JacksonGenerator
is ...
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.
Maybe I can describe this in a better way.
val JSON_GENERATOR_IGNORE_NULL_FIELDS = | ||
buildConf("spark.sql.jsonGenerator.ignoreNullFields") | ||
.doc("If false, JacksonGenerator will generate null for null fields in Struct.") | ||
.stringConf |
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.
Why is it a string conf? shouldn't it be a boolean conf?
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.
It is only used in JasonOptions, where we need string value, and use toBoolean to get boolean conf
val gen = new JacksonGenerator(dataType, writer, allowNullOption) | ||
gen.write(input) | ||
gen.flush() | ||
assert(writer.toString === """{"a":{"b":null}}""") |
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.
nit but I would call close
with a try-catch for a best practice in a followup.
cc @sameeragarwal (I vaguely remember, long time ago, your colleague opened a PR to support this case before) |
@stczwd can you make a followup to address the comments above? |
Yeah, I'll make it |
…elds in json generating # What changes were proposed in this pull request? Add description for ignoreNullFields, which is commited in #26098 , in DataFrameWriter and readwriter.py. Enable user to use ignoreNullFields in pyspark. ### Does this PR introduce any user-facing change? No ### How was this patch tested? run unit tests Closes #26227 from stczwd/json-generator-doc. Authored-by: stczwd <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]>
Hi, |
To make it sure, I triggered the snapshot publishing one minute ago. |
Thank you @dongjoon-hyun For instance, I am looking for: For version https://repository.apache.org/content/repositories/snapshots/org/apache/spark/spark-catalyst_2.11/ I tried 2.11 v 3.0.0 and that is unreleased version. I also do not understand how Best Regards, |
Can you please also release 3.0.0 for 2.11 as I am stuck with 2.11 version? |
AFAIK scala 2.11 has been dropped in Spark 3.0, cc @srowen |
This isn't quite the place to discuss, but yes there is no Scala 2.11 support in Spark 3.0. |
Ok, so keeping context – is there any chance this "issue" can be fixed in Spark 2? I don't know who is in the lead of this. However, I know that there are currently not many modules supporting Scala 2.12 (spark-streaming-kafka_2.11, spark-csv_2.11, lot of connectors,...), and that makes the transition between Spark 2 and 3 difficult. This situation throws me into the category of "Spark self workaround developers" to keep Sparking in our project. Is there any plan, recommendation and thread to this topic you can redirect me to? Thank you all for your time. |
All of Spark has supported 2.12 since 2.4.0, not sure what you mean. |
For instance, https://github.com/databricks/spark-csv doesn't seem to support 2.12 version yet. Maybe I just do not understand concept. But I suppose that we cannot combine 2.11 and 2.12 modules. Another example is https://github.com/memsql/memsql-spark-connector |
Per the repo, that was long ago pushed into Spark 2.x. CSV parsing is part of Spark and Spark supports 2.12. Third-party packages -- who knows. That memsql one maybe doesn't, but it also hasn't been updated since 2.0.x, so I don't even know if it works on 2.4 + Scala 2.11. All the less on Spark 3.0, likely. As we'll support scala 2.13 in Spark 3.x at some nearish point, we really can't keep supporting 2.11, which is EOL anyway. (Any further discussion -> use [email protected]) |
* [SPARK-29444] Add configuration to support JacksonGenrator to keep fields with null values As mentioned in jira, sometimes we need to be able to support the retention of null columns when writing JSON. For example, sparkmagic(used widely in jupyter with livy) will generate sql query results based on DataSet.toJSON and parse JSON to pandas DataFrame to display. If there is a null column, it is easy to have some column missing or even the query result is empty. The loss of the null column in the first row, may cause parsing exceptions or loss of entire column data. Example in spark-shell. scala> spark.sql("select null as a, 1 as b").toJSON.collect.foreach(println) {"b":1} scala> spark.sql("set spark.sql.jsonGenerator.struct.ignore.null=false") res2: org.apache.spark.sql.DataFrame = [key: string, value: string] scala> spark.sql("select null as a, 1 as b").toJSON.collect.foreach(println) {"a":null,"b":1} Add new test to JacksonGeneratorSuite Lead-authored-by: stczwd <[email protected]> Co-authored-by: Jackey Lee <[email protected]> Signed-off-by: Wenchen Fan <[email protected]> (cherry picked from commit with id 78b0cbe) * [SPARK-29444][FOLLOWUP] add doc and python parameter for ignoreNullFields in json generating # What changes were proposed in this pull request? Add description for ignoreNullFields, which is commited in apache#26098 , in DataFrameWriter and readwriter.py. Enable user to use ignoreNullFields in pyspark. ### Does this PR introduce any user-facing change? No ### How was this patch tested? run unit tests Closes apache#26227 from stczwd/json-generator-doc. Authored-by: stczwd <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]> Co-authored-by: stczwd <[email protected]>
* [SPARK-29444] Add configuration to support JacksonGenrator to keep fields with null values As mentioned in jira, sometimes we need to be able to support the retention of null columns when writing JSON. For example, sparkmagic(used widely in jupyter with livy) will generate sql query results based on DataSet.toJSON and parse JSON to pandas DataFrame to display. If there is a null column, it is easy to have some column missing or even the query result is empty. The loss of the null column in the first row, may cause parsing exceptions or loss of entire column data. Example in spark-shell. scala> spark.sql("select null as a, 1 as b").toJSON.collect.foreach(println) {"b":1} scala> spark.sql("set spark.sql.jsonGenerator.struct.ignore.null=false") res2: org.apache.spark.sql.DataFrame = [key: string, value: string] scala> spark.sql("select null as a, 1 as b").toJSON.collect.foreach(println) {"a":null,"b":1} Add new test to JacksonGeneratorSuite Lead-authored-by: stczwd <[email protected]> Co-authored-by: Jackey Lee <[email protected]> Signed-off-by: Wenchen Fan <[email protected]> (cherry picked from commit with id 78b0cbe) * [SPARK-29444][FOLLOWUP] add doc and python parameter for ignoreNullFields in json generating # What changes were proposed in this pull request? Add description for ignoreNullFields, which is commited in apache#26098 , in DataFrameWriter and readwriter.py. Enable user to use ignoreNullFields in pyspark. ### Does this PR introduce any user-facing change? No ### How was this patch tested? run unit tests Closes apache#26227 from stczwd/json-generator-doc. Authored-by: stczwd <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]> Co-authored-by: stczwd <[email protected]>
Why are the changes needed?
As mentioned in jira, sometimes we need to be able to support the retention of null columns when writing JSON.
For example, sparkmagic(used widely in jupyter with livy) will generate sql query results based on DataSet.toJSON and parse JSON to pandas DataFrame to display. If there is a null column, it is easy to have some column missing or even the query result is empty. The loss of the null column in the first row, may cause parsing exceptions or loss of entire column data.
Does this PR introduce any user-facing change?
Example in spark-shell.
scala> spark.sql("select null as a, 1 as b").toJSON.collect.foreach(println)
{"b":1}
scala> spark.sql("set spark.sql.jsonGenerator.struct.ignore.null=false")
res2: org.apache.spark.sql.DataFrame = [key: string, value: string]
scala> spark.sql("select null as a, 1 as b").toJSON.collect.foreach(println)
{"a":null,"b":1}
How was this patch tested?
Add new test to JacksonGeneratorSuite