failfast generally means that you find problems sooner rather than later, and 
here, potentially, that your code runs but simply returns empty data without 
any obvious cue as to what is wrong.

As is always good in OSS, follow those stack trace links to see what they say:

        // Check whether the path exists if it is not a glob pattern.
        // For glob pattern, we do not check it because the glob pattern might 
only make sense
        // once the streaming job starts and some upstream source starts 
dropping data.

If you specify a glob pattern, you'll get the late check at the expense of the 
risk of that empty data source if the pattern is wrong. Something like 
"/var/log\s" would suffice, as the presence of the backslash is enough for 
SparkHadoopUtil.isGlobPath() to conclude that its something for the globber.


> On 8 Sep 2016, at 07:33, Jacek Laskowski <ja...@japila.pl> wrote:
> 
> Hi,
> 
> I'm wondering what's the rationale for checking the path option
> eagerly in FileStreamSource? My thinking is that until start is called
> there's no processing going on that is supposed to happen on executors
> (not the driver) with the path available.
> 
> I could (and perhaps should) use dfs but IMHO that just hides the real
> question of the text source eagerness.
> 
> Please help me understand the rationale of the choice. Thanks!
> 
> scala> spark.version
> res0: String = 2.1.0-SNAPSHOT
> 
> scala> spark.readStream.format("text").load("/var/logs")
> org.apache.spark.sql.AnalysisException: Path does not exist: /var/logs;
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.sourceSchema(DataSource.scala:229)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.sourceInfo$lzycompute(DataSource.scala:81)
>  at 
> org.apache.spark.sql.execution.datasources.DataSource.sourceInfo(DataSource.scala:81)
>  at 
> org.apache.spark.sql.execution.streaming.StreamingRelation$.apply(StreamingRelation.scala:30)
>  at 
> org.apache.spark.sql.streaming.DataStreamReader.load(DataStreamReader.scala:142)
>  at 
> org.apache.spark.sql.streaming.DataStreamReader.load(DataStreamReader.scala:153)
>  ... 48 elided
> 
> Pozdrawiam,
> Jacek Laskowski
> ----
> https://medium.com/@jaceklaskowski/
> Mastering Apache Spark 2.0 http://bit.ly/mastering-apache-spark
> Follow me at https://twitter.com/jaceklaskowski
> 
> ---------------------------------------------------------------------
> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
> 
> 


---------------------------------------------------------------------
To unsubscribe e-mail: dev-unsubscr...@spark.apache.org

Reply via email to