Skip to content
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

Closed
wants to merge 6 commits into from

Conversation

jackylee-ch
Copy link
Contributor

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

@jackylee-ch
Copy link
Contributor Author

@jackylee-ch
Copy link
Contributor Author

Previous discussion about this: SPARK-23773

@jackylee-ch
Copy link
Contributor Author

cc @cloud-fan

@dongjoon-hyun
Copy link
Member

Hi, @stczwd . Thank you for making a PR.
Does this PR resolve SPARK-23773 ? Otherwise, you need to file an independent JIRA issue for this.

@@ -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") {
Copy link
Member

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok

@jackylee-ch
Copy link
Contributor Author

Hi, @stczwd . Thank you for making a PR.
Does this PR resolve SPARK-23773 ? Otherwise, you need to file an independent JIRA issue for this.

Yes, this PR can resolve this problem

@jackylee-ch jackylee-ch changed the title Add configuration to support JacksonGenrator to keep fields with null values [SPARK-29444] Add configuration to support JacksonGenrator to keep fields with null values Oct 13, 2019
@jackylee-ch
Copy link
Contributor Author

cc @dongjoon-hyun @xuanyuanking @cloud-fan @rdblue
any other question blocking this merged in?

@rdblue
Copy link
Contributor

rdblue commented Oct 15, 2019

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
Copy link
Contributor

@cloud-fan cloud-fan Oct 16, 2019

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?

Copy link
Contributor Author

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

Copy link
Contributor

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?

Copy link
Contributor Author

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

Copy link
Contributor

@cloud-fan cloud-fan Oct 16, 2019

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?

Copy link
Contributor Author

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.

@cloud-fan
Copy link
Contributor

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}""")
Copy link
Contributor

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}}

Copy link
Contributor Author

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

@jackylee-ch
Copy link
Contributor Author

The change looks reasonable. Do you know why the json data source ignore null fields at the first place?

@cloud-fan I don't know about this. Any ideas?

@jackylee-ch
Copy link
Contributor Author

@cloud-fan @dongjoon-hyun
Any other questions about it?

@dilipbiswal
Copy link
Contributor

@stczwd I have a question. After this change when we write to the json file, we will be able to preserve null values. Right ?

scala> spark.sql("select null as a, 1 as b").write.format("json").save("/tmp/jsonfile")

@jackylee-ch
Copy link
Contributor Author

@dilipbiswal Yes, but case is not right. If you want to preserve null values, you must write like this to disable ignoreNullFields.

spark.sql("select null as a, 1 as b").write.format("json").option("ignoreNullFields", "false").save("/tmp/jsonfile")

@dilipbiswal
Copy link
Contributor

@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")
Copy link
Contributor

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

Copy link
Contributor Author

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
Copy link
Contributor

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
}

Copy link
Contributor Author

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.

@jackylee-ch
Copy link
Contributor Author

@cloud-fan Any more questions?

@cloud-fan
Copy link
Contributor

ok to test

@xuanyuanking
Copy link
Member

The code changes LGTM.
I suggest adding more context in the PR description, we should clarify this PR is to add config to control the behavior added in SPARK-23773.

@SparkQA
Copy link

SparkQA commented Oct 17, 2019

Test build #112218 has finished for PR 26098 at commit f8aea25.

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

@@ -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",
Copy link
Member

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

Copy link
Contributor Author

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.")
Copy link
Member

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 ...

Copy link
Contributor Author

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
Copy link
Member

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?

Copy link
Contributor Author

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}}""")
Copy link
Member

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.

@HyukjinKwon
Copy link
Member

cc @sameeragarwal (I vaguely remember, long time ago, your colleague opened a PR to support this case before)

@HyukjinKwon
Copy link
Member

@stczwd can you make a followup to address the comments above?

@jackylee-ch
Copy link
Contributor Author

Yeah, I'll make it

@jackylee-ch jackylee-ch deleted the json branch October 23, 2019 10:21
dongjoon-hyun pushed a commit that referenced this pull request Oct 24, 2019
…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]>
@zacikpetr
Copy link

Hi,
Is there some repository I can try this solution?
I would expect it here https://repository.apache.org/content/repositories/snapshots/
Thank you for your help.

@dongjoon-hyun
Copy link
Member

To make it sure, I triggered the snapshot publishing one minute ago.

@zacikpetr
Copy link

Thank you @dongjoon-hyun
Ok, now I am officially confused as I do not understand your build process.

For instance, I am looking for: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala, that should contain JSON_GENERATOR_IGNORE_NULL_FIELDS
Class should be part of spark-catalyst.

For version https://repository.apache.org/content/repositories/snapshots/org/apache/spark/spark-catalyst_2.11/
only 2.4.5-SNAPSHOT gets updated
For version https://repository.apache.org/content/repositories/snapshots/org/apache/spark/spark-catalyst_2.12/
only version 3.0.0-SNAPSHOT gets updated

I tried 2.11 v 3.0.0 and that is unreleased version.
So I try to switch to these updated versions.

I also do not understand how avro is connected to this.

Best Regards,

@zacikpetr
Copy link

Can you please also release 3.0.0 for 2.11 as I am stuck with 2.11 version?

@cloud-fan
Copy link
Contributor

cloud-fan commented Oct 28, 2019

AFAIK scala 2.11 has been dropped in Spark 3.0, cc @srowen

@srowen
Copy link
Member

srowen commented Oct 28, 2019

This isn't quite the place to discuss, but yes there is no Scala 2.11 support in Spark 3.0.

@zacikpetr
Copy link

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.

@srowen
Copy link
Member

srowen commented Oct 29, 2019

All of Spark has supported 2.12 since 2.4.0, not sure what you mean.

@zacikpetr
Copy link

zacikpetr commented Oct 29, 2019

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

@srowen
Copy link
Member

srowen commented Oct 29, 2019

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])

agirish pushed a commit to HPEEzmeral/apache-spark that referenced this pull request May 5, 2022
* [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]>
udaynpusa pushed a commit to mapr/spark that referenced this pull request Jan 30, 2024
* [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]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

10 participants