You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "SuYan (JIRA)" <ji...@apache.org> on 2016/04/21 16:01:25 UTC

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

SuYan created SPARK-14804:
-----------------------------

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



{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