[ https://issues.apache.org/jira/browse/SPARK-23053?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16353761#comment-16353761 ]
huangtengfei edited comment on SPARK-23053 at 2/6/18 11:48 AM: --------------------------------------------------------------- here is the stack trace of exception. {code:java} java.lang.ClassCastException: org.apache.spark.rdd.CheckpointRDDPartition cannot be cast to org.apache.spark.streaming.rdd.MapWithStateRDDPartition at org.apache.spark.streaming.rdd.MapWithStateRDD.compute(MapWithStateRDD.scala:152) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD$$anonfun$8.apply(RDD.scala:336) at org.apache.spark.rdd.RDD$$anonfun$8.apply(RDD.scala:334) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:957) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:948) at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:888) at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:948) at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:694) at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:334) at org.apache.spark.rdd.RDD.iterator(RDD.scala:285) {code} was (Author: ivoson): here is the stack trace of exception. java.lang.ClassCastException: org.apache.spark.rdd.CheckpointRDDPartition cannot be cast to org.apache.spark.streaming.rdd.MapWithStateRDDPartition at org.apache.spark.streaming.rdd.MapWithStateRDD.compute(MapWithStateRDD.scala:152) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD$$anonfun$8.apply(RDD.scala:336) at org.apache.spark.rdd.RDD$$anonfun$8.apply(RDD.scala:334) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:957) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:948) at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:888) at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:948) at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:694) at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:334) at org.apache.spark.rdd.RDD.iterator(RDD.scala:285) > taskBinarySerialization and task partitions calculate in > DagScheduler.submitMissingTasks should keep the same RDD checkpoint status > ----------------------------------------------------------------------------------------------------------------------------------- > > Key: SPARK-23053 > URL: https://issues.apache.org/jira/browse/SPARK-23053 > Project: Spark > Issue Type: Bug > Components: Scheduler, Spark Core > Affects Versions: 2.1.0 > Reporter: huangtengfei > Priority: Major > > When we run concurrent jobs using the same rdd which is marked to do > checkpoint. If one job has finished running the job, and start the process of > RDD.doCheckpoint, while another job is submitted, then submitStage and > submitMissingTasks will be called. In > [submitMissingTasks|https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L961], > will serialize taskBinaryBytes and calculate task partitions which are both > affected by the status of checkpoint, if the former is calculated before > doCheckpoint finished, while the latter is calculated after doCheckpoint > finished, when run task, rdd.compute will be called, for some rdds with > particular partition type such as > [MapWithStateRDD|https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala] > who will do partition type cast, will get a ClassCastException because the > part params is actually a CheckpointRDDPartition. > This error occurs because rdd.doCheckpoint occurs in the same thread that > called sc.runJob, while the task serialization occurs in the DAGSchedulers > event loop. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org