Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Incomplete
-
1.2.0, 1.3.1
-
None
-
Yarn client
Description
Running the following code:
val subgraph = graph.subgraph (
vpred = (id,article) => //working predicate)
).cache()
println( s"Subgraph contains ${subgraph.vertices.count} nodes and ${subgraph.edges.count} edges")
val prGraph = subgraph.staticPageRank(5).cache
val titleAndPrGraph = subgraph.outerJoinVertices(prGraph.vertices)
{ (v, title, rank) => (rank.getOrElse(0.0), title) }titleAndPrGraph.vertices.top(13)
{ Ordering.by((entry: (VertexId, (Double, _))) => entry._2._1) }.foreach(t => println(t._2._2._1 + ": " + t._2._1 + ", id:" + t._1))
Returns a graph with 5000 nodes and 4000 edges.
Then it crashes during the PageRank with the following:
15/01/29 05:51:07 INFO scheduler.TaskSetManager: Starting task 125.0 in stage 39.0 (TID 1808, *HIDDEN, PROCESS_LOCAL, 2059 bytes)
15/01/29 05:51:07 WARN scheduler.TaskSetManager: Lost task 107.0 in stage 39.0 (TID 1794, *HIDDEN): java.lang.ArrayIndexOutOfBoundsException: -1
at org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap$mcJI$sp.apply$mcJI$sp(GraphXPrimitiveKeyOpenHashMap.scala:64)
at org.apache.spark.graphx.impl.EdgePartition.updateVertices(EdgePartition.scala:91)
at org.apache.spark.graphx.impl.ReplicatedVertexView$$anonfun$2$$anonfun$apply$1.apply(ReplicatedVertexView.scala:75)
at org.apache.spark.graphx.impl.ReplicatedVertexView$$anonfun$2$$anonfun$apply$1.apply(ReplicatedVertexView.scala:73)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
at org.apache.spark.graphx.impl.EdgeRDDImpl$$anonfun$mapEdgePartitions$1.apply(EdgeRDDImpl.scala:110)
at org.apache.spark.graphx.impl.EdgeRDDImpl$$anonfun$mapEdgePartitions$1.apply(EdgeRDDImpl.scala:108)
at org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:601)
at org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:601)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
at org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:61)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:228)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
at org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:88)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
at org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:88)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
at org.apache.spark.rdd.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:88)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
at org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:61)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:228)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
at org.apache.spark.scheduler.Task.run(Task.scala:56)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)
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:744)
Attachments
Issue Links
- relates to
-
SPARK-1329 ArrayIndexOutOfBoundsException if graphx.Graph has more edge partitions than node partitions
- Resolved