You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by an...@apache.org on 2014/12/03 02:14:16 UTC

spark git commit: [SPARK-4672][GraphX]Non-transient PartitionsRDDs will lead to StackOverflow error

Repository: spark
Updated Branches:
  refs/heads/master fc0a1475e -> 17c162f66


[SPARK-4672][GraphX]Non-transient PartitionsRDDs will lead to StackOverflow error

The related JIRA is https://issues.apache.org/jira/browse/SPARK-4672

In a nutshell, if `val partitionsRDD` in EdgeRDDImpl and VertexRDDImpl are non-transient, the serialization chain can become very long in iterative algorithms and finally lead to the StackOverflow error. More details and explanation can be found in the JIRA.

Author: JerryLead <Je...@163.com>
Author: Lijie Xu <cs...@gmail.com>

Closes #3544 from JerryLead/my_graphX and squashes the following commits:

628f33c [JerryLead] set PartitionsRDD to be transient in EdgeRDDImpl and VertexRDDImpl
c0169da [JerryLead] Merge branch 'master' of https://github.com/apache/spark
52799e3 [Lijie Xu] Merge pull request #1 from apache/master


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/17c162f6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/17c162f6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/17c162f6

Branch: refs/heads/master
Commit: 17c162f6682520e6e2790626e37da3a074471793
Parents: fc0a147
Author: JerryLead <Je...@163.com>
Authored: Tue Dec 2 17:14:11 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Tue Dec 2 17:14:11 2014 -0800

----------------------------------------------------------------------
 .../src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala  | 2 +-
 .../main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala    | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/17c162f6/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
index 504559d..897c7ee 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
@@ -26,7 +26,7 @@ import org.apache.spark.storage.StorageLevel
 import org.apache.spark.graphx._
 
 class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] (
-    override val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])],
+    @transient override val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])],
     val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY)
   extends EdgeRDD[ED](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/17c162f6/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala
index c8898b1..9732c5b 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala
@@ -27,7 +27,7 @@ import org.apache.spark.storage.StorageLevel
 import org.apache.spark.graphx._
 
 class VertexRDDImpl[VD] private[graphx] (
-    val partitionsRDD: RDD[ShippableVertexPartition[VD]],
+    @transient val partitionsRDD: RDD[ShippableVertexPartition[VD]],
     val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY)
   (implicit override protected val vdTag: ClassTag[VD])
   extends VertexRDD[VD](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org