-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-5484][GraphX] Periodically do checkpoint in Pregel #15125
Changes from 7 commits
3834981
166fd6d
b119e4a
d183a7c
352dcb2
ad82e45
e786838
a25d00c
38e6238
f2efef6
194dc27
9d7e796
dae94aa
dd6c366
2639eb1
11bc349
9a6fd1f
5015b44
24d4ad6
ec62659
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -362,12 +362,14 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali | |
def pregel[A: ClassTag]( | ||
initialMsg: A, | ||
maxIterations: Int = Int.MaxValue, | ||
checkpointInterval: Int = 25, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. isn't it a breaking change to add a param into the middle of the list? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. also, why is this There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good point in both cases. I'm wondering if the periodic pregel checkpointing operation should be controlled by a config value instead. Suppose, for example, we create a config key There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 25 is the value in my test. I checked this value in LDA/ALS, looks like the default value is 10, change it to 10 ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Agree with @mallman , we don't need change the api interface if we use a config value to controll checkpoint interval. I will udpate the PR soon. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. About the default value, I think we should set it as a positive value to turn on checkpoint operation by default to avoid stackoverflow exception. To align with other implementations in spark, I would like to set 10 as default value. Please let me know if there is any thoughts about this. |
||
activeDirection: EdgeDirection = EdgeDirection.Either)( | ||
vprog: (VertexId, VD, A) => VD, | ||
sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], | ||
mergeMsg: (A, A) => A) | ||
: Graph[VD, ED] = { | ||
Pregel(graph, initialMsg, maxIterations, activeDirection)(vprog, sendMsg, mergeMsg) | ||
Pregel(graph, initialMsg, maxIterations, activeDirection, | ||
checkpointInterval)(vprog, sendMsg, mergeMsg) | ||
} | ||
|
||
/** | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,7 +19,10 @@ package org.apache.spark.graphx | |
|
||
import scala.reflect.ClassTag | ||
|
||
import org.apache.spark.graphx.util.PeriodicGraphCheckpointer | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.util.PeriodicRDDCheckpointer | ||
|
||
/** | ||
* Implements a Pregel-like bulk-synchronous message-passing API. | ||
|
@@ -113,7 +116,8 @@ object Pregel extends Logging { | |
(graph: Graph[VD, ED], | ||
initialMsg: A, | ||
maxIterations: Int = Int.MaxValue, | ||
activeDirection: EdgeDirection = EdgeDirection.Either) | ||
activeDirection: EdgeDirection = EdgeDirection.Either, | ||
checkpointInterval: Int = 25) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ditto, why |
||
(vprog: (VertexId, VD, A) => VD, | ||
sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], | ||
mergeMsg: (A, A) => A) | ||
|
@@ -123,16 +127,25 @@ object Pregel extends Logging { | |
s" but got ${maxIterations}") | ||
|
||
var g = graph.mapVertices((vid, vdata) => vprog(vid, vdata, initialMsg)).cache() | ||
val graphCheckpointer = new PeriodicGraphCheckpointer[VD, ED]( | ||
checkpointInterval, graph.vertices.sparkContext) | ||
graphCheckpointer.update(g) | ||
|
||
// compute the messages | ||
var messages = GraphXUtils.mapReduceTriplets(g, sendMsg, mergeMsg) | ||
var messages = GraphXUtils.mapReduceTriplets(g, sendMsg, mergeMsg).cache() | ||
val messageCheckpointer = new PeriodicRDDCheckpointer[(VertexId, A)]( | ||
checkpointInterval, graph.vertices.sparkContext) | ||
messageCheckpointer.update(messages.asInstanceOf[RDD[(VertexId, A)]]) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we need to update it before the loop? I think it should be enough to do the update in the loop. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree. What do you think, @dding3? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Actually, for the sake of simplicity and consistency, I'm going to suggest we keep the checkpointer update calls but remove all There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we need cache graph/messages here so they don't need to be computed again in the loop. I agree with you and I will keep the checkpointer update calls and remove all .cache calls. |
||
var activeMessages = messages.count() | ||
|
||
// Loop | ||
var prevG: Graph[VD, ED] = null | ||
var i = 0 | ||
while (activeMessages > 0 && i < maxIterations) { | ||
// Receive the messages and update the vertices. | ||
prevG = g | ||
g = g.joinVertices(messages)(vprog).cache() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. checkpointer will do persist. So I think we don't need to call cache here. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Actually, this may be more subtle than I thought. I'm going to think through this again. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok. I agree with your observation here, @viirya. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. OK |
||
graphCheckpointer.update(g) | ||
|
||
val oldMessages = messages | ||
// Send new messages, skipping edges where neither side received a message. We must cache | ||
|
@@ -143,6 +156,7 @@ object Pregel extends Logging { | |
// The call to count() materializes `messages` and the vertices of `g`. This hides oldMessages | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. should the comment here be updated? |
||
// (depended on by the vertices of g) and the vertices of prevG (depended on by oldMessages | ||
// and the vertices of g). | ||
messageCheckpointer.update(messages.asInstanceOf[RDD[(VertexId, A)]]) | ||
activeMessages = messages.count() | ||
|
||
logInfo("Pregel finished iteration " + i) | ||
|
@@ -155,6 +169,8 @@ object Pregel extends Logging { | |
i += 1 | ||
} | ||
messages.unpersist(blocking = false) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: Let's do There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Looks like unpersist is a protected method and we cannot access it from Pregel. Add a new public method to unpersist dataset to work around this. |
||
graphCheckpointer.deleteAllCheckpoints() | ||
messageCheckpointer.deleteAllCheckpoints() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. shouldn't this be inside a finally clause to make sure checkpoint data is cleaned up even in error cases? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think when there is an exception during training, if we keep the checkpoints, there is a chance for user to recover from it. I checked in RandomForest/GBT in spark, looks like they only delete the checkpoints when the training successful finished. |
||
g | ||
} // end of apply | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -15,11 +15,12 @@ | |
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.mllib.impl | ||
package org.apache.spark.graphx.util | ||
|
||
import org.apache.spark.SparkContext | ||
import org.apache.spark.graphx.Graph | ||
import org.apache.spark.storage.StorageLevel | ||
import org.apache.spark.util.PeriodicCheckpointer | ||
|
||
|
||
/** | ||
|
@@ -76,7 +77,7 @@ import org.apache.spark.storage.StorageLevel | |
* | ||
* TODO: Move this out of MLlib? | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This comment should be removed. |
||
*/ | ||
private[mllib] class PeriodicGraphCheckpointer[VD, ED]( | ||
private[spark] class PeriodicGraphCheckpointer[VD, ED]( | ||
checkpointInterval: Int, | ||
sc: SparkContext) | ||
extends PeriodicCheckpointer[Graph[VD, ED]](checkpointInterval, sc) { | ||
|
@@ -87,10 +88,7 @@ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( | |
|
||
override protected def persist(data: Graph[VD, ED]): Unit = { | ||
if (data.vertices.getStorageLevel == StorageLevel.NONE) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. isn't persist better? this could potentially support different storage level later There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We need to use There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
data.vertices.persist() | ||
} | ||
if (data.edges.getStorageLevel == StorageLevel.NONE) { | ||
data.edges.persist() | ||
data.persist() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. hmm, There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I enhanced the persistence tests in The graph's vertex and edge rdds are somewhat peculiar in that At any rate, getting the |
||
} | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path | |
|
||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is there a reason this test suite isn't moved into the GraphX codebase? |
||
import org.apache.spark.{SparkContext, SparkFunSuite} | ||
import org.apache.spark.graphx.{Edge, Graph} | ||
import org.apache.spark.graphx.util.PeriodicGraphCheckpointer | ||
import org.apache.spark.mllib.util.MLlibTestSparkContext | ||
import org.apache.spark.storage.StorageLevel | ||
import org.apache.spark.util.Utils | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,7 +23,7 @@ import org.apache.spark.{SparkContext, SparkFunSuite} | |
import org.apache.spark.mllib.util.MLlibTestSparkContext | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is there a reason this file isn't moved into the core codebase? |
||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.storage.StorageLevel | ||
import org.apache.spark.util.Utils | ||
import org.apache.spark.util.{PeriodicRDDCheckpointer, Utils} | ||
|
||
|
||
class PeriodicRDDCheckpointerSuite extends SparkFunSuite with MLlibTestSparkContext { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -46,7 +46,12 @@ object MimaExcludes { | |
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.streaming.scheduler.StreamingListener.onStreamingStarted"), | ||
|
||
// [SPARK-19148][SQL] do not expose the external table concept in Catalog | ||
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.createTable") | ||
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.createTable"), | ||
|
||
// SPARK-5484 Periodically do checkpoint in Pregel | ||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.graphx.Pregel.apply"), | ||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.graphx.GraphOps.pregel"), | ||
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.graphx.GraphOps.pregel$default$3") | ||
) | ||
|
||
// Exclude rules for 2.1.x | ||
|
@@ -932,7 +937,7 @@ object MimaExcludes { | |
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.numTrees"), | ||
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setFeatureSubsetStrategy"), | ||
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.numTrees"), | ||
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setFeatureSubsetStrategy") | ||
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setFeatureSubsetStrategy") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. remove trailing whitespace There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. huh, why scala style checker doesn't complain about this. |
||
) | ||
} | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it's PeriodicRDDCheckpointer, shouldn't this be in the
org.apache.spark.rdd.util
namespace?