[ https://issues.apache.org/jira/browse/SPARK-23148?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17023935#comment-17023935 ]
Jorge Machado edited comment on SPARK-23148 at 1/27/20 7:22 AM: ---------------------------------------------------------------- Hi [~hyukjin.kwon] and [~henryr] , I have the same problem if I create a custom data source ``` class ImageFileValidator extends FileFormat with DataSourceRegister with Serializable ``` So the Problem Needs to be in some other places. Here my trace: I created https://issues.apache.org/jira/browse/SPARK-30647 {code:java} org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 2.0 failed 1 times, most recent failure: Lost task 1.0 in stage 2.0 (TID 213, localhost, executor driver): java.io.FileNotFoundException: File file:somePath/0019_leftImg8%20bit.png does not exist It is possible the underlying files have been updated. You can explicitly invalidate the cache in Spark by running 'REFRESH TABLE tableName' command in SQL or by recreating the Dataset/DataFrame involved. at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:127) at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:177) at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$13$$anon$1.hasNext(WholeStageCodegenExec.scala:636) at org.apache.spark.sql.execution.columnar.CachedRDDBuilder$$anonfun$1$$anon$1.hasNext(InMemoryRelation.scala:125) at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221) at org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:299) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1165) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) {code} was (Author: jomach): Hi [~hyukjin.kwon] and [~henryr] , I have the same problem if I create a custom data source ``` class ImageFileValidator extends FileFormat with DataSourceRegister with Serializable ``` So the Problem Needs to be in some other places. Here my trace: {code:java} org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 2.0 failed 1 times, most recent failure: Lost task 1.0 in stage 2.0 (TID 213, localhost, executor driver): java.io.FileNotFoundException: File file:somePath/0019_leftImg8%20bit.png does not exist It is possible the underlying files have been updated. You can explicitly invalidate the cache in Spark by running 'REFRESH TABLE tableName' command in SQL or by recreating the Dataset/DataFrame involved. at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:127) at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:177) at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$13$$anon$1.hasNext(WholeStageCodegenExec.scala:636) at org.apache.spark.sql.execution.columnar.CachedRDDBuilder$$anonfun$1$$anon$1.hasNext(InMemoryRelation.scala:125) at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221) at org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:299) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1165) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) {code} > spark.read.csv with multiline=true gives FileNotFoundException if path > contains spaces > -------------------------------------------------------------------------------------- > > Key: SPARK-23148 > URL: https://issues.apache.org/jira/browse/SPARK-23148 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 2.3.0 > Reporter: Bogdan Raducanu > Assignee: Henry Robinson > Priority: Major > Fix For: 2.3.0 > > > Repro code: > {code:java} > spark.range(10).write.csv("/tmp/a b c/a.csv") > spark.read.option("multiLine", false).csv("/tmp/a b c/a.csv").count > 10 > spark.read.option("multiLine", true).csv("/tmp/a b c/a.csv").count > java.io.FileNotFoundException: File > file:/tmp/a%20b%20c/a.csv/part-00000-cf84f9b2-5fe6-4f54-a130-a1737689db00-c000.csv > does not exist > {code} > Trying to manually escape fails in a different place: > {code} > spark.read.option("multiLine", true).csv("/tmp/a%20b%20c/a.csv").count > org.apache.spark.sql.AnalysisException: Path does not exist: > file:/tmp/a%20b%20c/a.csv; > at > org.apache.spark.sql.execution.datasources.DataSource$.org$apache$spark$sql$execution$datasources$DataSource$$checkAndGlobPathIfNecessary(DataSource.scala:683) > at > org.apache.spark.sql.execution.datasources.DataSource$$anonfun$15.apply(DataSource.scala:387) > at > org.apache.spark.sql.execution.datasources.DataSource$$anonfun$15.apply(DataSource.scala:387) > at > scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) > at > scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) > at scala.collection.immutable.List.foreach(List.scala:381) > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org