This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 6a59fba  [SPARK-37575][SQL] null values should be saved as nothing 
rather than quoted empty Strings "" by default settings
6a59fba is described below

commit 6a59fba248359fb2614837fe8781dc63ac8fdc4c
Author: wayneguow <guo...@gmail.com>
AuthorDate: Tue Dec 14 11:26:34 2021 +0300

    [SPARK-37575][SQL] null values should be saved as nothing rather than 
quoted empty Strings "" by default settings
    
    ### What changes were proposed in this pull request?
    Fix the bug that null values are saved as quoted empty strings "" (as the 
same as empty strings) rather than nothing by default csv settings since Spark 
2.4.
    
    ### Why are the changes needed?
    
    This is an unexpected bug, if don't fix it,  we still can't distinguish 
null values and empty strings in saved csv files.
    
    As mentioned in [spark sql migration 
guide](https://spark.apache.org/docs/latest/sql-migration-guide.html#upgrading-from-spark-sql-23-to-24)(2.3=>2.4),
 empty strings are saved as quoted empty string "", null values as saved as 
nothing since Spark 2.4.
    
    > Since Spark 2.4, empty strings are saved as quoted empty strings "". In 
version 2.3 and earlier, empty strings are equal to null values and do not 
reflect to any characters in saved CSV files. For example, the row of "a", 
null, "", 1 was written as a,,,1. Since Spark 2.4, the same row is saved as 
a,,"",1. To restore the previous behavior, set the CSV option emptyValue to 
empty (not quoted) string.
    
    But actually, we found that null values are also saved as quoted empty 
strings "" as the same as empty strings.
    
    For codes follows:
    ```scala
    Seq(("Tesla", null.asInstanceOf[String], ""))
      .toDF("make", "comment", "blank")
      .coalesce(1)
      .write.csv(path)
    ```
    
    actual results:
    >Tesla,"",""
    
    expected results:
    >Tesla,,""
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes, if this bug has been fixed, the output of null values would been 
changed to nothing rather than quoted empty strings "".
    
    But, users can set nullValue to "\\"\\""(same as emptyValueInWrite's 
default value) to restore the previous behavior since 2.4.
    
    ### How was this patch tested?
    
    Adding a test case.
    
    Closes #34853 from wayneguow/SPARK-37575.
    
    Lead-authored-by: wayneguow <guo...@gmail.com>
    Co-authored-by: Wayne Guo <guo...@gmail.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 .../apache/spark/sql/catalyst/csv/UnivocityGenerator.scala  |  2 --
 .../spark/sql/execution/datasources/csv/CSVSuite.scala      | 13 ++++++++++++-
 2 files changed, 12 insertions(+), 3 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
index 10cccd5..9d65824 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
@@ -94,8 +94,6 @@ class UnivocityGenerator(
     while (i < row.numFields) {
       if (!row.isNullAt(i)) {
         values(i) = valueConverters(i).apply(row, i)
-      } else {
-        values(i) = options.nullValue
       }
       i += 1
     }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index 8c8079f..c7328d9 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
@@ -805,6 +805,17 @@ abstract class CSVSuite
     }
   }
 
+  test("SPARK-37575: null values should be saved as nothing rather than " +
+    "quoted empty Strings \"\" with default settings") {
+    withTempPath { path =>
+      Seq(("Tesla", null: String, ""))
+        .toDF("make", "comment", "blank")
+        .write
+        .csv(path.getCanonicalPath)
+      checkAnswer(spark.read.text(path.getCanonicalPath), Row("Tesla,,\"\""))
+    }
+  }
+
   test("save csv with compression codec option") {
     withTempDir { dir =>
       val csvDir = new File(dir, "csv").getCanonicalPath
@@ -1769,7 +1780,7 @@ abstract class CSVSuite
         (1, "John Doe"),
         (2, "-"),
         (3, "-"),
-        (4, "-")
+        (4, null)
       ).toDF("id", "name")
 
       checkAnswer(computed, expected)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to