The input directory does need to be visible from the driver process, since
FileStreamSource does its polling from the driver. FileStreamSource creates
a Dataset for each microbatch.

I suppose the type-inference-time check for the presence of the input
directory could be moved to the FileStreamSource's initialization. But if
the directory isn't there when the source is being created, it probably
won't be there when the source is instantiated.

Fred

On Wed, Sep 7, 2016 at 11:33 PM, 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
>
>

Reply via email to