You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "Hisoka-X (via GitHub)" <gi...@apache.org> on 2023/09/18 12:27:10 UTC

[GitHub] [spark] Hisoka-X opened a new pull request, #42979: [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error

Hisoka-X opened a new pull request, #42979:
URL: https://github.com/apache/spark/pull/42979

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   Fix ignoreCorruptFiles with multiline CSV/JSON will report error, it would be like:
   ```log
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 4940.0 failed 4 times, most recent failure: Lost task 0.3 in stage 4940.0 (TID 4031) (10.68.177.106 executor 0): com.univocity.parsers.common.TextParsingException: java.lang.IllegalStateException - Error reading from input
   Parser Configuration: CsvParserSettings:
   	Auto configuration enabled=true
   	Auto-closing enabled=true
   	Autodetect column delimiter=false
   	Autodetect quotes=false
   	Column reordering enabled=true
   	Delimiters for detection=null
   	Empty value=
   	Escape unquoted values=false
   	Header extraction enabled=null
   	Headers=null
   	Ignore leading whitespaces=false
   	Ignore leading whitespaces in quotes=false
   	Ignore trailing whitespaces=false
   	Ignore trailing whitespaces in quotes=false
   	Input buffer size=1048576
   	Input reading on separate thread=false
   	Keep escape sequences=false
   	Keep quotes=false
   	Length of content displayed on error=1000
   	Line separator detection enabled=true
   	Maximum number of characters per column=-1
   	Maximum number of columns=20480
   	Normalize escaped line separators=true
   	Null value=
   	Number of records to read=all
   	Processor=none
   	Restricting data in exceptions=false
   	RowProcessor error handler=null
   	Selected fields=none
   	Skip bits as whitespace=true
   	Skip empty lines=true
   	Unescaped quote handling=STOP_AT_DELIMITERFormat configuration:
   	CsvFormat:
   		Comment character=#
   		Field delimiter=,
   		Line separator (normalized)=\n
   		Line separator sequence=\n
   		Quote character="
   		Quote escape character=\
   		Quote escape escape character=null
   Internal state when error was thrown: line=0, column=0, record=0
   	at com.univocity.parsers.common.AbstractParser.handleException(AbstractParser.java:402)
   	at com.univocity.parsers.common.AbstractParser.beginParsing(AbstractParser.java:277)
   	at com.univocity.parsers.common.AbstractParser.beginParsing(AbstractParser.java:843)
   	at org.apache.spark.sql.catalyst.csv.UnivocityParser$$anon$1.<init>(UnivocityParser.scala:463)
   	at org.apache.spark.sql.catalyst.csv.UnivocityParser$.convertStream(UnivocityParser.scala:46... 
   ```
   Because multiline CSV/JSON use `BinaryFileRDD` not `FileScanRDD`. Unlike `FileScanRDD`, when met corrupt files will check `ignoreCorruptFiles` config to avoid report IOException, `BinaryFileRDD` will not report error because it return normal `PortableDataStream`. So we should catch it when infer schema lambda function.
   
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   Fix the bug when use mulitline mode with ignoreCorruptFiles config.
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   add new test.
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   No
   <!--
   If generative AI tooling has been used in the process of authoring this patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1360500616


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala:
##########
@@ -1913,6 +1913,22 @@ abstract class JsonSuite
     }
   }
 
+  test("SPARK-45035: json enable ignoreCorruptFiles with multi-Line") {

Review Comment:
   Addressed all.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1360439004


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +100,14 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if options.ignoreMissingFiles =>
+            logWarning(s"Skipped missing file", e)
+            Some(StructType(Nil))
+          // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+          case e: FileNotFoundException if !options.ignoreMissingFiles => throw e
+          case e: IOException if options.ignoreCorruptFiles =>

Review Comment:
   `RuntimeException` already be catched in https://github.com/apache/spark/pull/42979/files/942b7c38a277a1b38b85a64ad940b56528ae8a03#diff-774d08eb04cd18039c576c7e23609430476d3dd2668535f0432f04b65b8ab234R92.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #42979: [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1331553477


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -190,12 +191,19 @@ object MultiLineCSVDataSource extends CSVDataSource {
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
     csv.flatMap { lines =>
-      val path = new Path(lines.getPath())
-      UnivocityParser.tokenizeStream(
-        CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
-        shouldDropHeader = false,
-        new CsvParser(parsedOptions.asParserSettings),
-        encoding = parsedOptions.charset)
+      try {
+        val path = new Path(lines.getPath())
+        UnivocityParser.tokenizeStream(
+          CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
+          shouldDropHeader = false,
+          new CsvParser(parsedOptions.asParserSettings),
+          encoding = parsedOptions.charset)
+      } catch {
+        case e @ (_: RuntimeException | _: IOException) if parsedOptions.ignoreCorruptFiles =>
+          logWarning(
+            s"Skipped the rest of the content in the corrupted file: ${lines.getPath()}", e)

Review Comment:
   How about change it to this?
   ```scala
           case e: TextParsingException if parsedOptions.ignoreCorruptFiles
             && e.getCause.getCause.isInstanceOf[EOFException] =>
   ```
   @HyukjinKwon 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1361947740


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -53,6 +54,9 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
     isParsing = true,
     forTimestampNTZ = true)
 
+  private val ignoreCorruptFiles = options.ignoreCorruptFiles
+  private val ignoreMissingFiles = options.ignoreMissingFiles

Review Comment:
   refer https://github.com/apache/spark/pull/42979#discussion_r1360256140 cc @MaxGekk 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1353811718


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -190,12 +191,19 @@ object MultiLineCSVDataSource extends CSVDataSource {
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
     csv.flatMap { lines =>
-      val path = new Path(lines.getPath())
-      UnivocityParser.tokenizeStream(
-        CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
-        shouldDropHeader = false,
-        new CsvParser(parsedOptions.asParserSettings),
-        encoding = parsedOptions.charset)
+      try {
+        val path = new Path(lines.getPath())
+        UnivocityParser.tokenizeStream(
+          CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
+          shouldDropHeader = false,
+          new CsvParser(parsedOptions.asParserSettings),
+          encoding = parsedOptions.charset)
+      } catch {
+        case e @ (_: RuntimeException | _: IOException) if parsedOptions.ignoreCorruptFiles =>
+          logWarning(
+            s"Skipped the rest of the content in the corrupted file: ${lines.getPath()}", e)

Review Comment:
   Addressed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1361964352


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala:
##########
@@ -1447,37 +1447,47 @@ abstract class CSVSuite
     }
   }
 
-  test("Enabling/disabling ignoreCorruptFiles") {
-    val inputFile = File.createTempFile("input-", ".gz")
-    try {
-      // Create a corrupt gzip file
-      val byteOutput = new ByteArrayOutputStream()
-      val gzip = new GZIPOutputStream(byteOutput)
-      try {
-        gzip.write(Array[Byte](1, 2, 3, 4))
-      } finally {
-        gzip.close()
-      }
-      val bytes = byteOutput.toByteArray
-      val o = new FileOutputStream(inputFile)
-      try {
-        // It's corrupt since we only write half of bytes into the file.
-        o.write(bytes.take(bytes.length / 2))
-      } finally {
-        o.close()
-      }
+  test("Enabling/disabling ignoreCorruptFiles/ignoreMissingFiles") {
+    withCorruptFile(inputFile => {
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
         val e = intercept[SparkException] {
           spark.read.csv(inputFile.toURI.toString).collect()
         }
         assert(e.getCause.getCause.isInstanceOf[EOFException])
         assert(e.getCause.getCause.getMessage === "Unexpected end of input stream")
+        val e2 = intercept[SparkException] {
+          spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+        }
+        assert(e2.getCause.getCause.getCause.isInstanceOf[EOFException])
+        assert(e2.getCause.getCause.getCause.getMessage === "Unexpected end of input stream")
       }
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") {
         assert(spark.read.csv(inputFile.toURI.toString).collect().isEmpty)
+        assert(spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+          .isEmpty)
+      }
+    })
+    withTempPath { dir =>
+      val csvPath = new Path(dir.getCanonicalPath, "csv")
+      val fs = csvPath.getFileSystem(spark.sessionState.newHadoopConf())
+
+      sampledTestData.write.csv(csvPath.toString)
+      val df = spark.read.option("multiLine", true).csv(csvPath.toString)
+      fs.delete(csvPath, true)
+      withSQLConf(SQLConf.IGNORE_MISSING_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          df.collect()
+        }
+        assert(e.getCause.isInstanceOf[SparkFileNotFoundException])
+        assert(e.getCause.getMessage.contains(".csv does not exist"))
+      }
+
+      sampledTestData.write.csv(csvPath.toString)
+      val df2 = spark.read.option("multiLine", true).csv(csvPath.toString)
+      fs.delete(csvPath, true)

Review Comment:
   I have tried it, but it seems that after executing `collect` once, the modified config does not take effect the second time. I haven't taken the time to find out the reason yet.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #42979:
URL: https://github.com/apache/spark/pull/42979#issuecomment-1767708571

   +1, LGTM. Merging to master.
   Thank you, @Hisoka-X and @cloud-fan @HyukjinKwon @beliefer for review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1363007575


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala:
##########
@@ -1447,37 +1447,47 @@ abstract class CSVSuite
     }
   }
 
-  test("Enabling/disabling ignoreCorruptFiles") {
-    val inputFile = File.createTempFile("input-", ".gz")
-    try {
-      // Create a corrupt gzip file
-      val byteOutput = new ByteArrayOutputStream()
-      val gzip = new GZIPOutputStream(byteOutput)
-      try {
-        gzip.write(Array[Byte](1, 2, 3, 4))
-      } finally {
-        gzip.close()
-      }
-      val bytes = byteOutput.toByteArray
-      val o = new FileOutputStream(inputFile)
-      try {
-        // It's corrupt since we only write half of bytes into the file.
-        o.write(bytes.take(bytes.length / 2))
-      } finally {
-        o.close()
-      }
+  test("Enabling/disabling ignoreCorruptFiles/ignoreMissingFiles") {
+    withCorruptFile(inputFile => {
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
         val e = intercept[SparkException] {
           spark.read.csv(inputFile.toURI.toString).collect()
         }
         assert(e.getCause.getCause.isInstanceOf[EOFException])
         assert(e.getCause.getCause.getMessage === "Unexpected end of input stream")
+        val e2 = intercept[SparkException] {
+          spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+        }
+        assert(e2.getCause.getCause.getCause.isInstanceOf[EOFException])
+        assert(e2.getCause.getCause.getCause.getMessage === "Unexpected end of input stream")
       }
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") {
         assert(spark.read.csv(inputFile.toURI.toString).collect().isEmpty)
+        assert(spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+          .isEmpty)
+      }
+    })
+    withTempPath { dir =>
+      val csvPath = new Path(dir.getCanonicalPath, "csv")
+      val fs = csvPath.getFileSystem(spark.sessionState.newHadoopConf())
+
+      sampledTestData.write.csv(csvPath.toString)
+      val df = spark.read.option("multiLine", true).csv(csvPath.toString)
+      fs.delete(csvPath, true)
+      withSQLConf(SQLConf.IGNORE_MISSING_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          df.collect()
+        }
+        assert(e.getCause.isInstanceOf[SparkFileNotFoundException])
+        assert(e.getCause.getMessage.contains(".csv does not exist"))
+      }
+
+      sampledTestData.write.csv(csvPath.toString)
+      val df2 = spark.read.option("multiLine", true).csv(csvPath.toString)
+      fs.delete(csvPath, true)

Review Comment:
   OK.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk closed pull request #42979: [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error
URL: https://github.com/apache/spark/pull/42979


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1362020082


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +102,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>
+            logWarning("Skipped missing file", e)
+            Some(StructType(Nil))
+          case e: FileNotFoundException if ignoreMissingFiles => throw e
+          case e @ (_: IOException | _: RuntimeException) if ignoreCorruptFiles =>

Review Comment:
   It is not for code format. It is an advice for follow catch rules of `FileScanRDD`. The purpose is to align with `FileScanRDD`'s processing logic on corrupt files (PS: To be honest, I'm not sure when the RuntimeException will occur, just to align the catch range of `FileScanRDD`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1362097871


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +102,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>
+            logWarning("Skipped missing file", e)
+            Some(StructType(Nil))
+          case e: FileNotFoundException if ignoreMissingFiles => throw e

Review Comment:
   isn't it dead code?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1362103791


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +102,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>
+            logWarning("Skipped missing file", e)
+            Some(StructType(Nil))
+          case e: FileNotFoundException if ignoreMissingFiles => throw e

Review Comment:
   oh, my mistake. This is a mistake when doing refactor. Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1360590413


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +100,14 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if options.ignoreMissingFiles =>
+            logWarning(s"Skipped missing file", e)
+            Some(StructType(Nil))
+          // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+          case e: FileNotFoundException if !options.ignoreMissingFiles => throw e
+          case e: IOException if options.ignoreCorruptFiles =>

Review Comment:
   I feel like JSON/CSV infer code should be similar to `FileScanRDD` and `ignoreCorruptFiles` should impact on errors caused by `RuntimeException`. What happens if put `RuntimeException` in the case? :
   ```scala
                     case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles =>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #42979: [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1329759589


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -190,12 +191,19 @@ object MultiLineCSVDataSource extends CSVDataSource {
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
     csv.flatMap { lines =>
-      val path = new Path(lines.getPath())
-      UnivocityParser.tokenizeStream(
-        CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
-        shouldDropHeader = false,
-        new CsvParser(parsedOptions.asParserSettings),
-        encoding = parsedOptions.charset)
+      try {
+        val path = new Path(lines.getPath())
+        UnivocityParser.tokenizeStream(
+          CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
+          shouldDropHeader = false,
+          new CsvParser(parsedOptions.asParserSettings),
+          encoding = parsedOptions.charset)
+      } catch {
+        case e @ (_: RuntimeException | _: IOException) if parsedOptions.ignoreCorruptFiles =>
+          logWarning(
+            s"Skipped the rest of the content in the corrupted file: ${lines.getPath()}", e)

Review Comment:
   Should reduce the scope of catching errors? I follow the catch range of FileScanRDD to determine whether it is a corrupted file.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #42979:
URL: https://github.com/apache/spark/pull/42979#issuecomment-1753060632

   @Hisoka-X Could you rebase this on the recent master Scala 2.13 + Java 17.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1362004408


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +102,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>
+            logWarning("Skipped missing file", e)
+            Some(StructType(Nil))
+          case e: FileNotFoundException if ignoreMissingFiles => throw e
+          case e @ (_: IOException | _: RuntimeException) if ignoreCorruptFiles =>

Review Comment:
   I understand that is an advice about code format. I am interested in what `RuntimeException` will happen.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1361948334


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +102,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>
+            logWarning("Skipped missing file", e)
+            Some(StructType(Nil))
+          case e: FileNotFoundException if ignoreMissingFiles => throw e
+          case e @ (_: IOException | _: RuntimeException) if ignoreCorruptFiles =>

Review Comment:
   refer https://github.com/apache/spark/pull/42979#discussion_r1360625931



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1360603779


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +100,14 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if options.ignoreMissingFiles =>
+            logWarning(s"Skipped missing file", e)
+            Some(StructType(Nil))
+          // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+          case e: FileNotFoundException if !options.ignoreMissingFiles => throw e
+          case e: IOException if options.ignoreCorruptFiles =>

Review Comment:
   I mean remove it from the case:
   ```scala
             case e @ (_: JsonProcessingException | _: MalformedInputException) =>
               handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, e)
   ...
   ```
   and put it there:
   ```scala
             case e @ (_: IOException | _: RuntimeException) if options.ignoreCorruptFiles =>
               logWarning("Skipped the rest of the content in the corrupted file", e)
               Some(StructType(Nil))
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1361865120


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala:
##########
@@ -1447,37 +1447,47 @@ abstract class CSVSuite
     }
   }
 
-  test("Enabling/disabling ignoreCorruptFiles") {
-    val inputFile = File.createTempFile("input-", ".gz")
-    try {
-      // Create a corrupt gzip file
-      val byteOutput = new ByteArrayOutputStream()
-      val gzip = new GZIPOutputStream(byteOutput)
-      try {
-        gzip.write(Array[Byte](1, 2, 3, 4))
-      } finally {
-        gzip.close()
-      }
-      val bytes = byteOutput.toByteArray
-      val o = new FileOutputStream(inputFile)
-      try {
-        // It's corrupt since we only write half of bytes into the file.
-        o.write(bytes.take(bytes.length / 2))
-      } finally {
-        o.close()
-      }
+  test("Enabling/disabling ignoreCorruptFiles/ignoreMissingFiles") {
+    withCorruptFile(inputFile => {
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
         val e = intercept[SparkException] {
           spark.read.csv(inputFile.toURI.toString).collect()
         }
         assert(e.getCause.getCause.isInstanceOf[EOFException])
         assert(e.getCause.getCause.getMessage === "Unexpected end of input stream")
+        val e2 = intercept[SparkException] {
+          spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()

Review Comment:
   Because the input file is the same as line 1454. How can we determine if it's the option `multiLine` issue?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala:
##########
@@ -1447,37 +1447,47 @@ abstract class CSVSuite
     }
   }
 
-  test("Enabling/disabling ignoreCorruptFiles") {
-    val inputFile = File.createTempFile("input-", ".gz")
-    try {
-      // Create a corrupt gzip file
-      val byteOutput = new ByteArrayOutputStream()
-      val gzip = new GZIPOutputStream(byteOutput)
-      try {
-        gzip.write(Array[Byte](1, 2, 3, 4))
-      } finally {
-        gzip.close()
-      }
-      val bytes = byteOutput.toByteArray
-      val o = new FileOutputStream(inputFile)
-      try {
-        // It's corrupt since we only write half of bytes into the file.
-        o.write(bytes.take(bytes.length / 2))
-      } finally {
-        o.close()
-      }
+  test("Enabling/disabling ignoreCorruptFiles/ignoreMissingFiles") {
+    withCorruptFile(inputFile => {
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
         val e = intercept[SparkException] {
           spark.read.csv(inputFile.toURI.toString).collect()
         }
         assert(e.getCause.getCause.isInstanceOf[EOFException])
         assert(e.getCause.getCause.getMessage === "Unexpected end of input stream")
+        val e2 = intercept[SparkException] {
+          spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+        }
+        assert(e2.getCause.getCause.getCause.isInstanceOf[EOFException])
+        assert(e2.getCause.getCause.getCause.getMessage === "Unexpected end of input stream")
       }
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") {
         assert(spark.read.csv(inputFile.toURI.toString).collect().isEmpty)
+        assert(spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()

Review Comment:
   How can we determine if the option `multiLine` affects the behavior?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1361994747


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -53,6 +54,9 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
     isParsing = true,
     forTimestampNTZ = true)
 
+  private val ignoreCorruptFiles = options.ignoreCorruptFiles
+  private val ignoreMissingFiles = options.ignoreMissingFiles

Review Comment:
   I know that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1362514689


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -53,6 +54,9 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
     isParsing = true,
     forTimestampNTZ = true)
 
+  private val ignoreCorruptFiles = options.ignoreCorruptFiles
+  private val ignoreMissingFiles = options.ignoreMissingFiles

Review Comment:
   I would keep the same code structure as in FileScanRDD for consistency. Let's do that, and refactor later together with FileScanRDD.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1360246269


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +100,14 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if options.ignoreMissingFiles =>
+            logWarning(s"Skipped missing file", e)
+            Some(StructType(Nil))
+          // Throw FileNotFoundException even if `ignoreCorruptFiles` is true

Review Comment:
   The comment is useless, it just rephrases the code below.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +100,14 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if options.ignoreMissingFiles =>

Review Comment:
   Should we place `options.ignoreMissingFiles` to a `val` as we do in `FileScanRDD`?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -190,12 +191,24 @@ object MultiLineCSVDataSource extends CSVDataSource {
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
     csv.flatMap { lines =>
-      val path = new Path(lines.getPath())
-      UnivocityParser.tokenizeStream(
-        CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
-        shouldDropHeader = false,
-        new CsvParser(parsedOptions.asParserSettings),
-        encoding = parsedOptions.charset)
+      try {
+        val path = new Path(lines.getPath())
+        UnivocityParser.tokenizeStream(
+          CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
+          shouldDropHeader = false,
+          new CsvParser(parsedOptions.asParserSettings),
+          encoding = parsedOptions.charset)
+      } catch {
+        case e: FileNotFoundException if parsedOptions.ignoreMissingFiles =>

Review Comment:
   The same as in JSON datasource, can you put the option to a `val`?
   https://github.com/apache/spark/blob/7796d8a63318d560b08d4d6a8b4d68ea0112bd3e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala#L88-L89



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +100,14 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if options.ignoreMissingFiles =>
+            logWarning(s"Skipped missing file", e)
+            Some(StructType(Nil))
+          // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+          case e: FileNotFoundException if !options.ignoreMissingFiles => throw e
+          case e: IOException if options.ignoreCorruptFiles =>

Review Comment:
   How about `RuntimeException` like at https://github.com/apache/spark/blob/7796d8a63318d560b08d4d6a8b4d68ea0112bd3e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala#L262 ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -190,12 +191,24 @@ object MultiLineCSVDataSource extends CSVDataSource {
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
     csv.flatMap { lines =>
-      val path = new Path(lines.getPath())
-      UnivocityParser.tokenizeStream(
-        CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
-        shouldDropHeader = false,
-        new CsvParser(parsedOptions.asParserSettings),
-        encoding = parsedOptions.charset)
+      try {
+        val path = new Path(lines.getPath())
+        UnivocityParser.tokenizeStream(
+          CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
+          shouldDropHeader = false,
+          new CsvParser(parsedOptions.asParserSettings),
+          encoding = parsedOptions.charset)
+      } catch {
+        case e: FileNotFoundException if parsedOptions.ignoreMissingFiles =>
+          logWarning(s"Skipped missing file: ${lines.getPath()}", e)
+          Array.empty[Array[String]]
+        // Throw FileNotFoundException even if `ignoreCorruptFiles` is true

Review Comment:
   Please, remove the comment.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala:
##########
@@ -1913,6 +1913,22 @@ abstract class JsonSuite
     }
   }
 
+  test("SPARK-45035: json enable ignoreCorruptFiles with multi-Line") {

Review Comment:
   Could you add a test for ignoreMissingFiles, please.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala:
##########
@@ -1448,37 +1447,25 @@ abstract class CSVSuite
   }
 
   test("Enabling/disabling ignoreCorruptFiles") {

Review Comment:
   Could you add a test for `ignoreMissingFiles`, please.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1360594941


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +100,14 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if options.ignoreMissingFiles =>
+            logWarning(s"Skipped missing file", e)
+            Some(StructType(Nil))
+          // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+          case e: FileNotFoundException if !options.ignoreMissingFiles => throw e
+          case e: IOException if options.ignoreCorruptFiles =>

Review Comment:
   It never reach here if we catch one `RuntimeException`. Because it will be catched by https://github.com/apache/spark/pull/42979/files/942b7c38a277a1b38b85a64ad940b56528ae8a03#diff-774d08eb04cd18039c576c7e23609430476d3dd2668535f0432f04b65b8ab234R92. So it would be useless code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1360553910


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +103,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>
+            logWarning(s"Skipped missing file", e)

Review Comment:
   addressed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1361872531


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala:
##########
@@ -1447,37 +1447,47 @@ abstract class CSVSuite
     }
   }
 
-  test("Enabling/disabling ignoreCorruptFiles") {
-    val inputFile = File.createTempFile("input-", ".gz")
-    try {
-      // Create a corrupt gzip file
-      val byteOutput = new ByteArrayOutputStream()
-      val gzip = new GZIPOutputStream(byteOutput)
-      try {
-        gzip.write(Array[Byte](1, 2, 3, 4))
-      } finally {
-        gzip.close()
-      }
-      val bytes = byteOutput.toByteArray
-      val o = new FileOutputStream(inputFile)
-      try {
-        // It's corrupt since we only write half of bytes into the file.
-        o.write(bytes.take(bytes.length / 2))
-      } finally {
-        o.close()
-      }
+  test("Enabling/disabling ignoreCorruptFiles/ignoreMissingFiles") {
+    withCorruptFile(inputFile => {
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
         val e = intercept[SparkException] {
           spark.read.csv(inputFile.toURI.toString).collect()
         }
         assert(e.getCause.getCause.isInstanceOf[EOFException])
         assert(e.getCause.getCause.getMessage === "Unexpected end of input stream")
+        val e2 = intercept[SparkException] {
+          spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+        }
+        assert(e2.getCause.getCause.getCause.isInstanceOf[EOFException])
+        assert(e2.getCause.getCause.getCause.getMessage === "Unexpected end of input stream")
       }
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") {
         assert(spark.read.csv(inputFile.toURI.toString).collect().isEmpty)
+        assert(spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()

Review Comment:
   How can we determine if the option `multiLine` affects the behavior?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala:
##########
@@ -1447,37 +1447,47 @@ abstract class CSVSuite
     }
   }
 
-  test("Enabling/disabling ignoreCorruptFiles") {
-    val inputFile = File.createTempFile("input-", ".gz")
-    try {
-      // Create a corrupt gzip file
-      val byteOutput = new ByteArrayOutputStream()
-      val gzip = new GZIPOutputStream(byteOutput)
-      try {
-        gzip.write(Array[Byte](1, 2, 3, 4))
-      } finally {
-        gzip.close()
-      }
-      val bytes = byteOutput.toByteArray
-      val o = new FileOutputStream(inputFile)
-      try {
-        // It's corrupt since we only write half of bytes into the file.
-        o.write(bytes.take(bytes.length / 2))
-      } finally {
-        o.close()
-      }
+  test("Enabling/disabling ignoreCorruptFiles/ignoreMissingFiles") {
+    withCorruptFile(inputFile => {
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
         val e = intercept[SparkException] {
           spark.read.csv(inputFile.toURI.toString).collect()
         }
         assert(e.getCause.getCause.isInstanceOf[EOFException])
         assert(e.getCause.getCause.getMessage === "Unexpected end of input stream")
+        val e2 = intercept[SparkException] {
+          spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()

Review Comment:
   Because the input file is the same as line 1454. How can we determine if it's the option `multiLine` issue?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1362038088


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala:
##########
@@ -1913,6 +1913,44 @@ abstract class JsonSuite
     }
   }
 
+  test("SPARK-45035: json enable ignoreCorruptFiles/ignoreMissingFiles with multi-Line") {
+    withCorruptFile(inputFile => {
+      withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          spark.read.option("multiLine", true).json(inputFile.toURI.toString).collect()

Review Comment:
   oh, sorry. I misread ignoreCorruptFiles and ignoreMissingFiles. Added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #42979: [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1329742029


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -190,12 +191,19 @@ object MultiLineCSVDataSource extends CSVDataSource {
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
     csv.flatMap { lines =>
-      val path = new Path(lines.getPath())
-      UnivocityParser.tokenizeStream(
-        CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
-        shouldDropHeader = false,
-        new CsvParser(parsedOptions.asParserSettings),
-        encoding = parsedOptions.charset)
+      try {
+        val path = new Path(lines.getPath())
+        UnivocityParser.tokenizeStream(
+          CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
+          shouldDropHeader = false,
+          new CsvParser(parsedOptions.asParserSettings),
+          encoding = parsedOptions.charset)
+      } catch {
+        case e @ (_: RuntimeException | _: IOException) if parsedOptions.ignoreCorruptFiles =>
+          logWarning(
+            s"Skipped the rest of the content in the corrupted file: ${lines.getPath()}", e)

Review Comment:
   I think the cause here might not be the ignored files?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on PR #42979:
URL: https://github.com/apache/spark/pull/42979#issuecomment-1753094389

   > @Hisoka-X Could you rebase this on the recent master Scala 2.13 + Java 17.
   
   Done


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1352776874


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -190,12 +191,19 @@ object MultiLineCSVDataSource extends CSVDataSource {
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
     csv.flatMap { lines =>
-      val path = new Path(lines.getPath())
-      UnivocityParser.tokenizeStream(
-        CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
-        shouldDropHeader = false,
-        new CsvParser(parsedOptions.asParserSettings),
-        encoding = parsedOptions.charset)
+      try {
+        val path = new Path(lines.getPath())
+        UnivocityParser.tokenizeStream(
+          CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
+          shouldDropHeader = false,
+          new CsvParser(parsedOptions.asParserSettings),
+          encoding = parsedOptions.charset)
+      } catch {
+        case e @ (_: RuntimeException | _: IOException) if parsedOptions.ignoreCorruptFiles =>
+          logWarning(
+            s"Skipped the rest of the content in the corrupted file: ${lines.getPath()}", e)

Review Comment:
   Actually, we divide the space of error to two areas:
   1. File not found if we catch the exception `FileNotFoundException`,
   2. Corrupted if any others happens: `IOException` or `RuntimeException`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1361910476


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -53,6 +54,9 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
     isParsing = true,
     forTimestampNTZ = true)
 
+  private val ignoreCorruptFiles = options.ignoreCorruptFiles

Review Comment:
   `options.ignoreCorruptFiles` used only once, please remove the variable.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -53,6 +54,9 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
     isParsing = true,
     forTimestampNTZ = true)
 
+  private val ignoreCorruptFiles = options.ignoreCorruptFiles
+  private val ignoreMissingFiles = options.ignoreMissingFiles

Review Comment:
   `options.ignoreMissingFile`s is a constant, so please remove the variable too.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -189,13 +190,26 @@ object MultiLineCSVDataSource extends CSVDataSource {
       inputPaths: Seq[FileStatus],
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
+    val ignoreCorruptFiles = parsedOptions.ignoreCorruptFiles
+    val ignoreMissingFiles = parsedOptions.ignoreMissingFiles
     csv.flatMap { lines =>
-      val path = new Path(lines.getPath())
-      UnivocityParser.tokenizeStream(
-        CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
-        shouldDropHeader = false,
-        new CsvParser(parsedOptions.asParserSettings),
-        encoding = parsedOptions.charset)
+      try {
+        val path = new Path(lines.getPath())
+        UnivocityParser.tokenizeStream(
+          CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
+          shouldDropHeader = false,
+          new CsvParser(parsedOptions.asParserSettings),
+          encoding = parsedOptions.charset)
+      } catch {
+        case e: FileNotFoundException if ignoreMissingFiles =>
+          logWarning(s"Skipped missing file: ${lines.getPath()}", e)
+          Array.empty[Array[String]]
+        case e: FileNotFoundException if !ignoreMissingFiles => throw e
+        case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles =>

Review Comment:
   What's the cause of RuntimeException now?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala:
##########
@@ -1913,6 +1913,44 @@ abstract class JsonSuite
     }
   }
 
+  test("SPARK-45035: json enable ignoreCorruptFiles/ignoreMissingFiles with multi-Line") {
+    withCorruptFile(inputFile => {
+      withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          spark.read.option("multiLine", true).json(inputFile.toURI.toString).collect()
+        }
+        assert(e.getCause.isInstanceOf[EOFException])
+        assert(e.getCause.getMessage === "Unexpected end of input stream")
+      }
+      withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") {
+        assert(spark.read.option("multiLine", true).json(inputFile.toURI.toString).collect()
+          .isEmpty)
+      }
+    })
+    withTempPath { dir =>
+      val jsonPath = new Path(dir.getCanonicalPath, "json")
+      val fs = jsonPath.getFileSystem(spark.sessionState.newHadoopConf())
+
+      sampledTestData.write.json(jsonPath.toString)
+      val df = spark.read.option("multiLine", true).json(jsonPath.toString)
+      fs.delete(jsonPath, true)
+      withSQLConf(SQLConf.IGNORE_MISSING_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          df.collect()
+        }
+        assert(e.getCause.isInstanceOf[SparkFileNotFoundException])
+        assert(e.getCause.getMessage.contains(".json does not exist"))
+      }
+
+      sampledTestData.write.json(jsonPath.toString)
+      val df2 = spark.read.option("multiLine", true).json(jsonPath.toString)
+      fs.delete(jsonPath, true)

Review Comment:
   ditto.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala:
##########
@@ -1913,6 +1913,44 @@ abstract class JsonSuite
     }
   }
 
+  test("SPARK-45035: json enable ignoreCorruptFiles/ignoreMissingFiles with multi-Line") {
+    withCorruptFile(inputFile => {
+      withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          spark.read.option("multiLine", true).json(inputFile.toURI.toString).collect()

Review Comment:
   Is there a lack of testing for `TextInputJsonDataSource`?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -189,13 +190,26 @@ object MultiLineCSVDataSource extends CSVDataSource {
       inputPaths: Seq[FileStatus],
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
+    val ignoreCorruptFiles = parsedOptions.ignoreCorruptFiles

Review Comment:
   ditto.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +102,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>
+            logWarning("Skipped missing file", e)
+            Some(StructType(Nil))
+          case e: FileNotFoundException if ignoreMissingFiles => throw e
+          case e @ (_: IOException | _: RuntimeException) if ignoreCorruptFiles =>

Review Comment:
   What's the cause of RuntimeException?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -189,13 +190,26 @@ object MultiLineCSVDataSource extends CSVDataSource {
       inputPaths: Seq[FileStatus],
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
+    val ignoreCorruptFiles = parsedOptions.ignoreCorruptFiles
+    val ignoreMissingFiles = parsedOptions.ignoreMissingFiles

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala:
##########
@@ -1447,37 +1447,47 @@ abstract class CSVSuite
     }
   }
 
-  test("Enabling/disabling ignoreCorruptFiles") {
-    val inputFile = File.createTempFile("input-", ".gz")
-    try {
-      // Create a corrupt gzip file
-      val byteOutput = new ByteArrayOutputStream()
-      val gzip = new GZIPOutputStream(byteOutput)
-      try {
-        gzip.write(Array[Byte](1, 2, 3, 4))
-      } finally {
-        gzip.close()
-      }
-      val bytes = byteOutput.toByteArray
-      val o = new FileOutputStream(inputFile)
-      try {
-        // It's corrupt since we only write half of bytes into the file.
-        o.write(bytes.take(bytes.length / 2))
-      } finally {
-        o.close()
-      }
+  test("Enabling/disabling ignoreCorruptFiles/ignoreMissingFiles") {
+    withCorruptFile(inputFile => {
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
         val e = intercept[SparkException] {
           spark.read.csv(inputFile.toURI.toString).collect()
         }
         assert(e.getCause.getCause.isInstanceOf[EOFException])
         assert(e.getCause.getCause.getMessage === "Unexpected end of input stream")
+        val e2 = intercept[SparkException] {
+          spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+        }
+        assert(e2.getCause.getCause.getCause.isInstanceOf[EOFException])
+        assert(e2.getCause.getCause.getCause.getMessage === "Unexpected end of input stream")
       }
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") {
         assert(spark.read.csv(inputFile.toURI.toString).collect().isEmpty)
+        assert(spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+          .isEmpty)
+      }
+    })
+    withTempPath { dir =>
+      val csvPath = new Path(dir.getCanonicalPath, "csv")
+      val fs = csvPath.getFileSystem(spark.sessionState.newHadoopConf())
+
+      sampledTestData.write.csv(csvPath.toString)
+      val df = spark.read.option("multiLine", true).csv(csvPath.toString)
+      fs.delete(csvPath, true)
+      withSQLConf(SQLConf.IGNORE_MISSING_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          df.collect()
+        }
+        assert(e.getCause.isInstanceOf[SparkFileNotFoundException])
+        assert(e.getCause.getMessage.contains(".csv does not exist"))
+      }
+
+      sampledTestData.write.csv(csvPath.toString)
+      val df2 = spark.read.option("multiLine", true).csv(csvPath.toString)
+      fs.delete(csvPath, true)

Review Comment:
   It seems we can reuse `df` here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1362099980


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +102,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>

Review Comment:
   It's a bit annoying that we need to handle corrupted/missing files in multiple places, but I don't have a good idea.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +102,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>

Review Comment:
   It's a bit annoying that we need to handle corrupted/missing files in multiple places, but I don't have a better idea.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1360615993


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +100,14 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if options.ignoreMissingFiles =>
+            logWarning(s"Skipped missing file", e)
+            Some(StructType(Nil))
+          // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+          case e: FileNotFoundException if !options.ignoreMissingFiles => throw e
+          case e: IOException if options.ignoreCorruptFiles =>

Review Comment:
   oh, I got it. But I'm not sure it would change behavior or not. Let me change it and see CI would report error or not.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1360538628


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +103,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>
+            logWarning(s"Skipped missing file", e)

Review Comment:
   Is s before "" needed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1360625931


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +100,14 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if options.ignoreMissingFiles =>
+            logWarning(s"Skipped missing file", e)
+            Some(StructType(Nil))
+          // Throw FileNotFoundException even if `ignoreCorruptFiles` is true
+          case e: FileNotFoundException if !options.ignoreMissingFiles => throw e
+          case e: IOException if options.ignoreCorruptFiles =>

Review Comment:
   From my understanding, we consider a JSON record as malformed when the JSON parser cannot parse its already retrieved input. So, it was able to read some text from a file but cannot parse. For instance, if we look at JSON parser exceptions:
   ```java
       public JsonParser createParser(InputStream in) throws IOException, JsonParseException {
           IOContext ctxt = _createContext(_createContentReference(in), false);
           return _createParser(_decorate(in, ctxt), ctxt);
       }
   ...
       public abstract JsonToken nextToken() throws IOException;
   ```
   JsonParseException (JsonProcessingException) + IOException
   
   So, the `RuntimeException` can come only for some corrupted files. cc @HyukjinKwon @cloud-fan 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] Hisoka-X commented on pull request #42979: [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on PR #42979:
URL: https://github.com/apache/spark/pull/42979#issuecomment-1723306751

   cc @HyukjinKwon 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #42979: [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1331553477


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -190,12 +191,19 @@ object MultiLineCSVDataSource extends CSVDataSource {
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
     csv.flatMap { lines =>
-      val path = new Path(lines.getPath())
-      UnivocityParser.tokenizeStream(
-        CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
-        shouldDropHeader = false,
-        new CsvParser(parsedOptions.asParserSettings),
-        encoding = parsedOptions.charset)
+      try {
+        val path = new Path(lines.getPath())
+        UnivocityParser.tokenizeStream(
+          CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
+          shouldDropHeader = false,
+          new CsvParser(parsedOptions.asParserSettings),
+          encoding = parsedOptions.charset)
+      } catch {
+        case e @ (_: RuntimeException | _: IOException) if parsedOptions.ignoreCorruptFiles =>
+          logWarning(
+            s"Skipped the rest of the content in the corrupted file: ${lines.getPath()}", e)

Review Comment:
   How change it to this?
   ```scala
           case e: TextParsingException if parsedOptions.ignoreCorruptFiles
             && e.getCause.getCause.isInstanceOf[EOFException] =>
   ```
   @HyukjinKwon 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1362026999


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala:
##########
@@ -1447,37 +1447,47 @@ abstract class CSVSuite
     }
   }
 
-  test("Enabling/disabling ignoreCorruptFiles") {
-    val inputFile = File.createTempFile("input-", ".gz")
-    try {
-      // Create a corrupt gzip file
-      val byteOutput = new ByteArrayOutputStream()
-      val gzip = new GZIPOutputStream(byteOutput)
-      try {
-        gzip.write(Array[Byte](1, 2, 3, 4))
-      } finally {
-        gzip.close()
-      }
-      val bytes = byteOutput.toByteArray
-      val o = new FileOutputStream(inputFile)
-      try {
-        // It's corrupt since we only write half of bytes into the file.
-        o.write(bytes.take(bytes.length / 2))
-      } finally {
-        o.close()
-      }
+  test("Enabling/disabling ignoreCorruptFiles/ignoreMissingFiles") {
+    withCorruptFile(inputFile => {
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
         val e = intercept[SparkException] {
           spark.read.csv(inputFile.toURI.toString).collect()
         }
         assert(e.getCause.getCause.isInstanceOf[EOFException])
         assert(e.getCause.getCause.getMessage === "Unexpected end of input stream")
+        val e2 = intercept[SparkException] {
+          spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+        }
+        assert(e2.getCause.getCause.getCause.isInstanceOf[EOFException])
+        assert(e2.getCause.getCause.getCause.getMessage === "Unexpected end of input stream")
       }
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") {
         assert(spark.read.csv(inputFile.toURI.toString).collect().isEmpty)
+        assert(spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+          .isEmpty)
+      }
+    })
+    withTempPath { dir =>
+      val csvPath = new Path(dir.getCanonicalPath, "csv")
+      val fs = csvPath.getFileSystem(spark.sessionState.newHadoopConf())
+
+      sampledTestData.write.csv(csvPath.toString)
+      val df = spark.read.option("multiLine", true).csv(csvPath.toString)
+      fs.delete(csvPath, true)
+      withSQLConf(SQLConf.IGNORE_MISSING_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          df.collect()
+        }
+        assert(e.getCause.isInstanceOf[SparkFileNotFoundException])
+        assert(e.getCause.getMessage.contains(".csv does not exist"))
+      }
+
+      sampledTestData.write.csv(csvPath.toString)
+      val df2 = spark.read.option("multiLine", true).csv(csvPath.toString)
+      fs.delete(csvPath, true)

Review Comment:
   Yes.
   ![image](https://github.com/apache/spark/assets/32387433/53596615-2cc0-4ac9-b91c-cf4dcf7a2890)
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1361951439


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala:
##########
@@ -1913,6 +1913,44 @@ abstract class JsonSuite
     }
   }
 
+  test("SPARK-45035: json enable ignoreCorruptFiles/ignoreMissingFiles with multi-Line") {
+    withCorruptFile(inputFile => {
+      withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          spark.read.option("multiLine", true).json(inputFile.toURI.toString).collect()

Review Comment:
   already tested in `FileBasedDataSourceSuite`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1362030220


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala:
##########
@@ -99,6 +102,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
             val wrappedCharException = new CharConversionException(msg)
             wrappedCharException.initCause(e)
             handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, wrappedCharException)
+          case e: FileNotFoundException if ignoreMissingFiles =>
+            logWarning("Skipped missing file", e)
+            Some(StructType(Nil))
+          case e: FileNotFoundException if ignoreMissingFiles => throw e
+          case e @ (_: IOException | _: RuntimeException) if ignoreCorruptFiles =>

Review Comment:
   Yeah. That advice just for follow the catch rules(Personally, it's related to code format).
   Because the origin code catches `RuntimeException`, ignore the comment above is also acceptable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1362013903


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala:
##########
@@ -1913,6 +1913,44 @@ abstract class JsonSuite
     }
   }
 
+  test("SPARK-45035: json enable ignoreCorruptFiles/ignoreMissingFiles with multi-Line") {
+    withCorruptFile(inputFile => {
+      withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          spark.read.option("multiLine", true).json(inputFile.toURI.toString).collect()

Review Comment:
   I can't find them. Could you paste the link?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala:
##########
@@ -1447,37 +1447,47 @@ abstract class CSVSuite
     }
   }
 
-  test("Enabling/disabling ignoreCorruptFiles") {
-    val inputFile = File.createTempFile("input-", ".gz")
-    try {
-      // Create a corrupt gzip file
-      val byteOutput = new ByteArrayOutputStream()
-      val gzip = new GZIPOutputStream(byteOutput)
-      try {
-        gzip.write(Array[Byte](1, 2, 3, 4))
-      } finally {
-        gzip.close()
-      }
-      val bytes = byteOutput.toByteArray
-      val o = new FileOutputStream(inputFile)
-      try {
-        // It's corrupt since we only write half of bytes into the file.
-        o.write(bytes.take(bytes.length / 2))
-      } finally {
-        o.close()
-      }
+  test("Enabling/disabling ignoreCorruptFiles/ignoreMissingFiles") {
+    withCorruptFile(inputFile => {
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") {
         val e = intercept[SparkException] {
           spark.read.csv(inputFile.toURI.toString).collect()
         }
         assert(e.getCause.getCause.isInstanceOf[EOFException])
         assert(e.getCause.getCause.getMessage === "Unexpected end of input stream")
+        val e2 = intercept[SparkException] {
+          spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+        }
+        assert(e2.getCause.getCause.getCause.isInstanceOf[EOFException])
+        assert(e2.getCause.getCause.getCause.getMessage === "Unexpected end of input stream")
       }
       withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") {
         assert(spark.read.csv(inputFile.toURI.toString).collect().isEmpty)
+        assert(spark.read.option("multiLine", true).csv(inputFile.toURI.toString).collect()
+          .isEmpty)
+      }
+    })
+    withTempPath { dir =>
+      val csvPath = new Path(dir.getCanonicalPath, "csv")
+      val fs = csvPath.getFileSystem(spark.sessionState.newHadoopConf())
+
+      sampledTestData.write.csv(csvPath.toString)
+      val df = spark.read.option("multiLine", true).csv(csvPath.toString)
+      fs.delete(csvPath, true)
+      withSQLConf(SQLConf.IGNORE_MISSING_FILES.key -> "false") {
+        val e = intercept[SparkException] {
+          df.collect()
+        }
+        assert(e.getCause.isInstanceOf[SparkFileNotFoundException])
+        assert(e.getCause.getMessage.contains(".csv does not exist"))
+      }
+
+      sampledTestData.write.csv(csvPath.toString)
+      val df2 = spark.read.option("multiLine", true).csv(csvPath.toString)
+      fs.delete(csvPath, true)

Review Comment:
   Have you ever tried deleting these.
   ```
         sampledTestData.write.csv(csvPath.toString)
          val df2 = spark.read.option("multiLine", true).csv(csvPath.toString)
          fs.delete(csvPath, true)
   ```
   and change the below code to
   ```
          withSQLConf(SQLConf.IGNORE_MISSING_FILES.key -> "true") {
            assert(df.collect().isEmpty)
          }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


Re: [PR] [SPARK-45035][SQL] Fix ignoreCorruptFiles with multiline CSV/JSON will report error [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #42979:
URL: https://github.com/apache/spark/pull/42979#discussion_r1352767119


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala:
##########
@@ -190,12 +191,19 @@ object MultiLineCSVDataSource extends CSVDataSource {
       parsedOptions: CSVOptions): StructType = {
     val csv = createBaseRdd(sparkSession, inputPaths, parsedOptions)
     csv.flatMap { lines =>
-      val path = new Path(lines.getPath())
-      UnivocityParser.tokenizeStream(
-        CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
-        shouldDropHeader = false,
-        new CsvParser(parsedOptions.asParserSettings),
-        encoding = parsedOptions.charset)
+      try {
+        val path = new Path(lines.getPath())
+        UnivocityParser.tokenizeStream(
+          CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path),
+          shouldDropHeader = false,
+          new CsvParser(parsedOptions.asParserSettings),
+          encoding = parsedOptions.charset)
+      } catch {
+        case e @ (_: RuntimeException | _: IOException) if parsedOptions.ignoreCorruptFiles =>
+          logWarning(
+            s"Skipped the rest of the content in the corrupted file: ${lines.getPath()}", e)

Review Comment:
   >  follow the catch range of FileScanRDD to determine whether it is a corrupted file.
   
   I think need to follow the cascade of catching exceptions. For example, `FileNotFoundException` is a sub-class of `IOException`, and it should be controlled by the separate flag: `ignoreMissingFiles`.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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