You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Martin Brišiak (JIRA)" <ji...@apache.org> on 2017/05/24 10:40:04 UTC

[jira] [Created] (SPARK-20865) caching dataset throws "Queries with streaming sources must be executed with writeStream.start()"

Martin Brišiak created SPARK-20865:
--------------------------------------

             Summary: caching dataset throws "Queries with streaming sources must be executed with writeStream.start()"
                 Key: SPARK-20865
                 URL: https://issues.apache.org/jira/browse/SPARK-20865
             Project: Spark
          Issue Type: Bug
          Components: SQL, Structured Streaming
    Affects Versions: 2.1.1, 2.1.0, 2.0.2
            Reporter: Martin Brišiak


SparkSession
  .builder
  .master("local[*]")
  .config("spark.sql.warehouse.dir", "C:/tmp/spark")
  .config("spark.sql.streaming.checkpointLocation", "C:/tmp/spark/spark-checkpoint")
  .appName("my-test")
  .getOrCreate
  .readStream
  .schema(schema)
  .json("src/test/data")
  .cache
  .writeStream
  .start
  .awaitTermination

While executing this sample in spark got error. Without the .cache option it worked as intended but with .cache option i got:

Exception in thread "main" org.apache.spark.sql.AnalysisException: Queries with streaming sources must be executed with writeStream.start();; FileSource[src/test/data] at org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker$.org$apache$spark$sql$catalyst$analysis$UnsupportedOperationChecker$$throwError(UnsupportedOperationChecker.scala:196) at org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker$$anonfun$checkForBatch$1.apply(UnsupportedOperationChecker.scala:35) at org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker$$anonfun$checkForBatch$1.apply(UnsupportedOperationChecker.scala:33) at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:128) at org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker$.checkForBatch(UnsupportedOperationChecker.scala:33) at org.apache.spark.sql.execution.QueryExecution.assertSupported(QueryExecution.scala:58) at org.apache.spark.sql.execution.QueryExecution.withCachedData$lzycompute(QueryExecution.scala:69) at org.apache.spark.sql.execution.QueryExecution.withCachedData(QueryExecution.scala:67) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:73) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:73) at org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:79) at org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:75) at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:84) at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:84) at org.apache.spark.sql.execution.CacheManager$$anonfun$cacheQuery$1.apply(CacheManager.scala:102) at org.apache.spark.sql.execution.CacheManager.writeLock(CacheManager.scala:65) at org.apache.spark.sql.execution.CacheManager.cacheQuery(CacheManager.scala:89) at org.apache.spark.sql.Dataset.persist(Dataset.scala:2479) at org.apache.spark.sql.Dataset.cache(Dataset.scala:2489) at org.me.App$.main(App.scala:23) at org.me.App.main(App.scala)



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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