You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Joseph K. Bradley (JIRA)" <ji...@apache.org> on 2017/01/04 19:19:58 UTC

[jira] [Commented] (SPARK-14804) Graph vertexRDD/EdgeRDD checkpoint results ClassCastException:

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

Joseph K. Bradley commented on SPARK-14804:
-------------------------------------------

I think this is a separate issue from local checkpointing.  The problem appears to be that Graph checkpointing incorrectly uses internal checkpointing APIs.

> Graph vertexRDD/EdgeRDD checkpoint results ClassCastException: 
> ---------------------------------------------------------------
>
>                 Key: SPARK-14804
>                 URL: https://issues.apache.org/jira/browse/SPARK-14804
>             Project: Spark
>          Issue Type: Bug
>          Components: GraphX
>    Affects Versions: 1.6.1
>            Reporter: SuYan
>            Priority: Minor
>
> {code}
>     graph3.vertices.checkpoint()
>     graph3.vertices.count()
>     graph3.vertices.map(_._2).count()
> {code}
> 16/04/21 21:04:43 WARN scheduler.TaskSetManager: Lost task 0.0 in stage 4.0 (TID 13, localhost): java.lang.ClassCastException: org.apache.spark.graphx.impl.ShippableVertexPartition cannot be cast to scala.Tuple2
> 	at com.xiaomi.infra.codelab.spark.Graph2$$anonfun$main$1.apply(Graph2.scala:80)
> 	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
> 	at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1597)
> 	at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1161)
> 	at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1161)
> 	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1863)
> 	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1863)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:91)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:219)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at java.lang.Thread.run(Thread.java:745)
> look at the code:
> {code}
>   private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] =
>   {
>     if (isCheckpointedAndMaterialized) {
>       firstParent[T].iterator(split, context)
>     } else {
>       compute(split, context)
>     }
>   }
>  private[spark] def isCheckpointedAndMaterialized: Boolean = isCheckpointed
>  override def isCheckpointed: Boolean = {
>    firstParent[(PartitionID, EdgePartition[ED, VD])].isCheckpointed
>  }
> {code}
> for VertexRDD or EdgeRDD, first parent is its partitionRDD  RDD[ShippableVertexPartition[VD]]/RDD[(PartitionID, EdgePartition[ED, VD])]
> 1. we call vertexRDD.checkpoint, it partitionRDD will checkpoint, so VertexRDD.isCheckpointedAndMaterialized=true.
> 2. then we call vertexRDD.iterator, because checkoint=true it called firstParent.iterator(which is not CheckpointRDD, actually is partitionRDD). 
>  
> so returned iterator is iterator[ShippableVertexPartition] not expect iterator[(VertexId, VD)]]



--
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