Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16976#discussion_r103060062
  
    --- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
 ---
    @@ -961,56 +978,135 @@ class CSVSuite extends QueryTest with 
SharedSQLContext with SQLTestUtils {
       }
     
       test("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` 
field") {
    -    val schema = new StructType().add("a", IntegerType).add("b", 
TimestampType)
    -    val df1 = spark
    -      .read
    -      .option("mode", "PERMISSIVE")
    -      .schema(schema)
    -      .csv(testFile(valueMalformedFile))
    -    checkAnswer(df1,
    -      Row(null, null) ::
    -      Row(1, java.sql.Date.valueOf("1983-08-04")) ::
    -      Nil)
    -
    -    // If `schema` has `columnNameOfCorruptRecord`, it should handle 
corrupt records
    -    val columnNameOfCorruptRecord = "_unparsed"
    -    val schemaWithCorrField1 = schema.add(columnNameOfCorruptRecord, 
StringType)
    -    val df2 = spark
    -      .read
    -      .option("mode", "PERMISSIVE")
    -      .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord)
    -      .schema(schemaWithCorrField1)
    -      .csv(testFile(valueMalformedFile))
    -    checkAnswer(df2,
    -      Row(null, null, "0,2013-111-11 12:13:14") ::
    -      Row(1, java.sql.Date.valueOf("1983-08-04"), null) ::
    -      Nil)
    -
    -    // We put a `columnNameOfCorruptRecord` field in the middle of a schema
    -    val schemaWithCorrField2 = new StructType()
    -      .add("a", IntegerType)
    -      .add(columnNameOfCorruptRecord, StringType)
    -      .add("b", TimestampType)
    -    val df3 = spark
    -      .read
    -      .option("mode", "PERMISSIVE")
    -      .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord)
    -      .schema(schemaWithCorrField2)
    -      .csv(testFile(valueMalformedFile))
    -    checkAnswer(df3,
    -      Row(null, "0,2013-111-11 12:13:14", null) ::
    -      Row(1, null, java.sql.Date.valueOf("1983-08-04")) ::
    -      Nil)
    -
    -    val errMsg = intercept[AnalysisException] {
    -      spark
    +    Seq(false, true).foreach { bool =>
    +      val schema = new StructType().add("a", IntegerType).add("b", 
TimestampType)
    +      val df1 = spark
    +        .read
    +        .option("mode", "PERMISSIVE")
    +        .option("wholeFile", bool)
    +        .schema(schema)
    +        .csv(testFile(valueMalformedFile))
    +      checkAnswer(df1,
    +        Row(null, null) ::
    +        Row(1, java.sql.Date.valueOf("1983-08-04")) ::
    +        Nil)
    +
    +      // If `schema` has `columnNameOfCorruptRecord`, it should handle 
corrupt records
    +      val columnNameOfCorruptRecord = "_unparsed"
    +      val schemaWithCorrField1 = schema.add(columnNameOfCorruptRecord, 
StringType)
    +      val df2 = spark
             .read
             .option("mode", "PERMISSIVE")
             .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord)
    -        .schema(schema.add(columnNameOfCorruptRecord, IntegerType))
    +        .option("wholeFile", bool)
    +        .schema(schemaWithCorrField1)
             .csv(testFile(valueMalformedFile))
    -        .collect
    -    }.getMessage
    -    assert(errMsg.startsWith("The field for corrupt records must be string 
type and nullable"))
    +      checkAnswer(df2,
    +        Row(null, null, "0,2013-111-11 12:13:14") ::
    +        Row(1, java.sql.Date.valueOf("1983-08-04"), null) ::
    +        Nil)
    +
    +      // We put a `columnNameOfCorruptRecord` field in the middle of a 
schema
    +      val schemaWithCorrField2 = new StructType()
    +        .add("a", IntegerType)
    +        .add(columnNameOfCorruptRecord, StringType)
    +        .add("b", TimestampType)
    +      val df3 = spark
    +        .read
    +        .option("mode", "PERMISSIVE")
    +        .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord)
    +        .option("wholeFile", bool)
    +        .schema(schemaWithCorrField2)
    +        .csv(testFile(valueMalformedFile))
    +      checkAnswer(df3,
    +        Row(null, "0,2013-111-11 12:13:14", null) ::
    +        Row(1, null, java.sql.Date.valueOf("1983-08-04")) ::
    +        Nil)
    +
    +      val errMsg = intercept[AnalysisException] {
    +        spark
    +          .read
    +          .option("mode", "PERMISSIVE")
    +          .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord)
    +          .option("wholeFile", bool)
    +          .schema(schema.add(columnNameOfCorruptRecord, IntegerType))
    +          .csv(testFile(valueMalformedFile))
    +          .collect
    +      }.getMessage
    +      assert(errMsg.startsWith("The field for corrupt records must be 
string type and nullable"))
    +    }
    +  }
    +
    +  test("SPARK-19610: Parse normal multi-line CSV files") {
    +    val primitiveFieldAndType = Seq(
    +      """"
    +        |string","integer
    +        |
    +        |
    +        |","long
    +        |
    +        |","bigInteger",double,boolean,null""".stripMargin,
    +      """"this is a
    +        |simple
    +        |string.","
    +        |
    +        |10","
    +        |21474836470","92233720368547758070","
    +        |
    +        |1.7976931348623157E308",true,""".stripMargin)
    +
    +    withTempPath { path =>
    +      
primitiveFieldAndType.toDF("value").coalesce(1).write.text(path.getAbsolutePath)
    +
    +      val df = spark.read
    +        .option("header", true)
    +        .option("wholeFile", true)
    +        .csv(path.getAbsolutePath)
    +
    +      // Check if headers have new lines in the names.
    +      val actualFields = df.schema.fieldNames.toSeq
    +      val expectedFields =
    +        Seq("\nstring", "integer\n\n\n", "long\n\n", "bigInteger", 
"double", "boolean", "null")
    +      assert(actualFields === expectedFields)
    +
    +      // Check if the rows have new lines in the values.
    +      val expected = Row(
    +        "this is a\nsimple\nstring.",
    +        "\n\n10",
    +        "\n21474836470",
    +        "92233720368547758070",
    +        "\n\n1.7976931348623157E308",
    +        "true",
    +         null)
    +      checkAnswer(df, expected)
    +
    +      val csvDir = new File(path.getAbsolutePath, "csv").getAbsolutePath
    +      val cleanCols = df.schema
    +        .map(f => regexp_replace(col(f.name), "[\\r\\n]", ""))
    +      val dfWithoutWhiteSpaces = df.select(cleanCols: _*)
    +      dfWithoutWhiteSpaces.write.csv(csvDir)
    --- End diff --
    
    I meant to test that we are able to read this back without `wholeFile` 
option if we remove all newlines.
    
    ```scala
    // Check if the rows are the same if we remove all white spaces.
    val readBack = spark.read.csv(csvDir)
    checkAnswer(dfWithoutWhiteSpaces, readBack)
    ```
    
    Let me clean up this one too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to