You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Jack Hu (JIRA)" <ji...@apache.org> on 2015/04/13 05:35:12 UTC

[jira] [Comment Edited] (SPARK-6847) Stack overflow on updateStateByKey which followed by a dstream with checkpoint set

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

Jack Hu edited comment on SPARK-6847 at 4/13/15 3:34 AM:
---------------------------------------------------------

Hi, [~sowen]

I tested more cases:
# only change the {{newlist.headOption.orElse(oldstate)}} to {{Some("a")}}, the issue still exists
# only change the streaming batch interval to {{2 seconds}}, keep the  {{newlist.headOption.orElse(oldstate)}} and checkpoint interval 10 seconds, the issue does not exist. 

So this issue may be related to the checkpoint interval and batch interval. 


was (Author: jhu):
Hi, [~sowen]

I tested more cases:
# only change the {{newlist.headOption.orElse(oldstate)}} to {{Some("a")}}, the issue still exists
# only change the streaming batch interval to {{2 seconds}}, keep the  {{newlist.headOption.orElse(oldstate)}} and checkpoint interval 10 seconds, the issue does not exist. 

So this issue may related to the checkpoint interval and batch interval. 

> 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
>            Reporter: Jack Hu
>              Labels: StackOverflowError, Streaming
>
> 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-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org