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

gurwls223 pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
     new 4ab3b11885f [SPARK-38955][SQL] Disable lineSep option in 'from_csv' 
and 'schema_of_csv'
4ab3b11885f is described below

commit 4ab3b11885fe6ead02c44b9a6281cd56ce14a9bc
Author: Hyukjin Kwon <gurwls...@apache.org>
AuthorDate: Fri Apr 22 11:43:46 2022 +0900

    [SPARK-38955][SQL] Disable lineSep option in 'from_csv' and 'schema_of_csv'
    
    This PR proposes to disable `lineSep` option in `from_csv` and 
`schema_of_csv` expression by setting Noncharacters according to [unicode 
specification](https://www.unicode.org/charts/PDF/UFFF0.pdf), `\UFFFF`. This 
can be used for the internal purpose in a program according to the 
specification.
    
    The Univocity parser does not allow to omit the line separator (from my 
code reading) so this approach was proposed.
    
    This specific code path is not affected by our `encoding` or `charset` 
option because Unicovity parser parses them as unicodes as are internally.
    
    Currently, this option is weirdly effective. See the example of `from_csv` 
as below:
    
    ```scala
    import org.apache.spark.sql.types._
    import org.apache.spark.sql.functions._
    
    Seq[String]("1,\n2,3,4,5").toDF.select(
      col("value"),
      from_csv(
        col("value"),
        StructType(Seq(StructField("a", LongType), StructField("b", StringType)
      )), Map[String,String]())).show()
    ```
    
    ```
    +-----------+---------------+
    |      value|from_csv(value)|
    +-----------+---------------+
    |1,\n2,3,4,5|      {1, null}|
    +-----------+---------------+
    ```
    
    `{1, null}` has to be `{1, \n2}`.
    
    The CSV expressions cannot easily make it supported because this option is 
plan-wise option that can change the number of returned rows; however, the 
expressions are designed to emit one row only whereas this option is easily 
effective in the scan plan with CSV data source. Therefore, we should disable 
this option.
    
    Yes, now the `lineSep` can be located in the output from `from_csv` and 
`schema_of_csv`.
    
    Manually tested, and unit test was added.
    
    Closes #36294 from HyukjinKwon/SPARK-38955.
    
    Authored-by: Hyukjin Kwon <gurwls...@apache.org>
    Signed-off-by: Hyukjin Kwon <gurwls...@apache.org>
    (cherry picked from commit f3cc2814d4bc585dad92c9eca9a593d1617d27e9)
    Signed-off-by: Hyukjin Kwon <gurwls...@apache.org>
    (cherry picked from commit d63e42d128b8814e885b86533f187724fbb7e9fd)
    Signed-off-by: Hyukjin Kwon <gurwls...@apache.org>
---
 .../spark/sql/catalyst/expressions/csvExpressions.scala  | 16 ++++++++++++++--
 .../scala/org/apache/spark/sql/CsvFunctionsSuite.scala   |  7 +++++++
 2 files changed, 21 insertions(+), 2 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala
index 79bbc103c92..1225c8620e2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala
@@ -98,8 +98,14 @@ case class CsvToStructs(
   val nameOfCorruptRecord = 
SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD)
 
   @transient lazy val parser = {
+    // 'lineSep' is a plan-wise option so we set a noncharacter, according to
+    // the unicode specification, which should not appear in Java's strings.
+    // See also SPARK-38955 and https://www.unicode.org/charts/PDF/UFFF0.pdf.
+    // scalastyle:off nonascii
+    val exprOptions = options ++ Map("lineSep" -> '\uFFFF'.toString)
+    // scalastyle:on nonascii
     val parsedOptions = new CSVOptions(
-      options,
+      exprOptions,
       columnPruning = true,
       defaultTimeZoneId = timeZoneId.get,
       defaultColumnNameOfCorruptRecord = nameOfCorruptRecord)
@@ -186,7 +192,13 @@ case class SchemaOfCsv(
   }
 
   override def eval(v: InternalRow): Any = {
-    val parsedOptions = new CSVOptions(options, true, "UTC")
+    // 'lineSep' is a plan-wise option so we set a noncharacter, according to
+    // the unicode specification, which should not appear in Java's strings.
+    // See also SPARK-38955 and https://www.unicode.org/charts/PDF/UFFF0.pdf.
+    // scalastyle:off nonascii
+    val exprOptions = options ++ Map("lineSep" -> '\uFFFF'.toString)
+    // scalastyle:on nonascii
+    val parsedOptions = new CSVOptions(exprOptions, true, "UTC")
     val parser = new CsvParser(parsedOptions.asParserSettings)
     val row = parser.parseLine(csv.toString)
     assert(row != null, "Parsed CSV record should not be null.")
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala
index 711a4bc3fdb..a66dcd9a7fe 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala
@@ -353,4 +353,11 @@ class CsvFunctionsSuite extends QueryTest with 
SharedSparkSession {
       }
     }
   }
+
+  test("SPARK-38955: disable lineSep option in from_csv and schema_of_csv") {
+    val df = Seq[String]("1,2\n2").toDF("csv")
+    val actual = df.select(from_csv(
+      $"csv", schema_of_csv("1,2\n2"), Map.empty[String, String].asJava))
+    checkAnswer(actual, Row(Row(1, "2\n2")))
+  }
 }


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

Reply via email to