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

gurwls223 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 bb4c4778713 [SPARK-39689][SQL] Support 2-chars `lineSep` in CSV 
datasource
bb4c4778713 is described below

commit bb4c4778713c7ba1ee92d0bb0763d7d3ce54374f
Author: yaohua <yaohua.z...@databricks.com>
AuthorDate: Thu Jul 7 15:22:06 2022 +0900

    [SPARK-39689][SQL] Support 2-chars `lineSep` in CSV datasource
    
    ### What changes were proposed in this pull request?
    Univocity parser allows to set line separator to 1 to 2 characters 
([code](https://github.com/uniVocity/univocity-parsers/blob/master/src/main/java/com/univocity/parsers/common/Format.java#L103)),
 CSV options should not block this usage 
([code](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala#L218)).
 This PR updates the requirement of `lineSep` accepting 1 or 2 characters in 
`CSVOptions`.
    
    Due to the limitation of supporting multi-chars `lineSep` within quotes, I 
propose to leave this feature undocumented and add a WARN message to users.
    
    ### Why are the changes needed?
    Unblock the usability of 2 characters `lineSep`.
    
    ### Does this PR introduce _any_ user-facing change?
    No - undocumented feature.
    
    ### How was this patch tested?
    New UT.
    
    Closes #37107 from Yaohua628/spark-39689.
    
    Lead-authored-by: yaohua <yaohua.z...@databricks.com>
    Co-authored-by: Yaohua Zhao <79476540+yaohua...@users.noreply.github.com>
    Signed-off-by: Hyukjin Kwon <gurwls...@apache.org>
---
 .../apache/spark/sql/catalyst/csv/CSVOptions.scala |  6 +++-
 .../sql/execution/datasources/csv/CSVSuite.scala   | 35 ++++++++++++++++++++++
 2 files changed, 40 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala
index 9daa50ba5a4..3e92c3d25eb 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala
@@ -215,7 +215,11 @@ class CSVOptions(
    */
   val lineSeparator: Option[String] = parameters.get("lineSep").map { sep =>
     require(sep.nonEmpty, "'lineSep' cannot be an empty string.")
-    require(sep.length == 1, "'lineSep' can contain only 1 character.")
+    // Intentionally allow it up to 2 for Window's CRLF although multiple
+    // characters have an issue with quotes. This is intentionally 
undocumented.
+    require(sep.length <= 2, "'lineSep' can contain only 1 character.")
+    if (sep.length == 2) logWarning("It is not recommended to set 'lineSep' " +
+      "with 2 characters due to the limitation of supporting multi-char 
'lineSep' within quotes.")
     sep
   }
 
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 62dccaad1dd..bf92ffcf465 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
@@ -34,6 +34,7 @@ import org.apache.commons.lang3.exception.ExceptionUtils
 import org.apache.commons.lang3.time.FastDateFormat
 import org.apache.hadoop.io.SequenceFile.CompressionType
 import org.apache.hadoop.io.compress.GzipCodec
+import org.apache.logging.log4j.Level
 
 import org.apache.spark.{SparkConf, SparkException, TestUtils}
 import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Encoders, 
QueryTest, Row}
@@ -2296,6 +2297,40 @@ abstract class CSVSuite
     assert(errMsg2.contains("'lineSep' can contain only 1 character"))
   }
 
+  Seq(true, false).foreach { multiLine =>
+    test(s"""lineSep with 2 chars when multiLine set to $multiLine""") {
+      Seq("\r\n", "||", "|").foreach { newLine =>
+        val logAppender = new LogAppender("lineSep WARN logger")
+        withTempDir { dir =>
+          val inputData = if (multiLine) {
+            s"""name,"i am the${newLine} 
column1"${newLine}jack,30${newLine}tom,18"""
+          } else {
+            s"name,age${newLine}jack,30${newLine}tom,18"
+          }
+          Files.write(new File(dir, "/data.csv").toPath, inputData.getBytes())
+          withLogAppender(logAppender) {
+            val df = spark.read
+              .options(
+                Map("header" -> "true", "multiLine" -> multiLine.toString, 
"lineSep" -> newLine))
+              .csv(dir.getCanonicalPath)
+            // Due to the limitation of Univocity parser:
+            // multiple chars of newlines cannot be properly handled when they 
exist within quotes.
+            // Leave 2-char lineSep as an undocumented features and logWarn 
user
+            if (newLine != "||" || !multiLine) {
+              checkAnswer(df, Seq(Row("jack", "30"), Row("tom", "18")))
+            }
+            if (newLine.length == 2) {
+              val message = "It is not recommended to set 'lineSep' with 2 
characters due to"
+              assert(logAppender.loggingEvents.exists(
+                e => e.getLevel == Level.WARN && 
e.getMessage.getFormattedMessage.contains(message)
+              ))
+            }
+          }
+        }
+      }
+    }
+  }
+
   test("SPARK-26208: write and read empty data to csv file with headers") {
     withTempPath { path =>
       val df1 = spark.range(10).repartition(2).filter(_ < 
0).map(_.toString).toDF


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

Reply via email to