[
https://issues.apache.org/jira/browse/SPARK-6847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14493517#comment-14493517
]
Jack Hu commented on SPARK-6847:
--------------------------------
Here is the part of the stack (Full stack at:
https://gist.github.com/jhu-chang/38a6c052aff1d666b785)
{quote}
15/04/14 11:28:20 [Executor task launch worker-1] ERROR
org.apache.spark.executor.Executor: Exception in task 1.0 in stage 27554.0 (TID
3801)
java.lang.StackOverflowError
at
java.io.ObjectStreamClass.setPrimFieldValues(ObjectStreamClass.java:1243)
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1984)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1918)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1993)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1918)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371)
at scala.collection.immutable.$colon$colon.readObject(List.scala:362)
at sun.reflect.GeneratedMethodAccessor3.invoke(Unknown Source)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:483)
at
java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1896)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1993)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1918)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1993)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1918)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371)
at scala.collection.immutable.$colon$colon.readObject(List.scala:366)
at sun.reflect.GeneratedMethodAccessor3.invoke(Unknown Source)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:483)
at
java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1896)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1993)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1918)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1993)
at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1918)
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371)
at scala.collection.immutable.$colon$colon.readObject(List.scala:362)
at sun.reflect.GeneratedMethodAccessor3.invoke(Unknown Source)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:483)
{quote}
> 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: [email protected]
For additional commands, e-mail: [email protected]