[ 
https://issues.apache.org/jira/browse/SPARK-6847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15126928#comment-15126928
 ] 

Shixiong Zhu commented on SPARK-6847:
-------------------------------------

As my PR changed internal semantics, it's only merged to master branch (2.0.0). 

For pre 2.0.0, you may need to trigger the checkpoint by yourself. E.g., for 
{{updateStateByKey().filter().updateStateByKey()}}, you can update to 
{{dstream.updateStateByKey().count(); 
dstream.updateStateByKey().filter().updateStateByKey()}} to trigger the 
checkpoint for the first "updateStateByKey".

> Stack overflow on updateStateByKey which followed by a dstream with 
> checkpoint set
> ----------------------------------------------------------------------------------
>
>                 Key: SPARK-6847
>                 URL: https://issues.apache.org/jira/browse/SPARK-6847
>             Project: Spark
>          Issue Type: Bug
>          Components: Streaming
>    Affects Versions: 1.3.0, 1.4.1, 1.5.2, 1.6.0
>            Reporter: Jack Hu
>            Assignee: Shixiong Zhu
>            Priority: Critical
>              Labels: StackOverflowError, Streaming
>             Fix For: 2.0.0
>
>
> The issue happens with the following sample code: uses {{updateStateByKey}} 
> followed by a {{map}} with checkpoint interval 10 seconds
> {code}
>     val sparkConf = new SparkConf().setAppName("test")
>     val streamingContext = new StreamingContext(sparkConf, Seconds(10))
>     streamingContext.checkpoint("""checkpoint""")
>     val source = streamingContext.socketTextStream("localhost", 9999)
>     val updatedResult = source.map(
>         (1,_)).updateStateByKey(
>             (newlist : Seq[String], oldstate : Option[String]) =>     
> newlist.headOption.orElse(oldstate))
>     updatedResult.map(_._2)
>     .checkpoint(Seconds(10))
>     .foreachRDD((rdd, t) => {
>       println("Deep: " + rdd.toDebugString.split("\n").length)
>       println(t.toString() + ": " + rdd.collect.length)
>     })
>     streamingContext.start()
>     streamingContext.awaitTermination()
> {code}
> From the output, we can see that the dependency will be increasing time over 
> time, the {{updateStateByKey}} never get check-pointed,  and finally, the 
> stack overflow will happen. 
> Note:
> * The rdd in {{updatedResult.map(_._2)}} get check-pointed in this case, but 
> not the {{updateStateByKey}} 
> * If remove the {{checkpoint(Seconds(10))}} from the map result ( 
> {{updatedResult.map(_._2)}} ), the stack overflow will not happen



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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

Reply via email to