Skip to content

Commit c581593

Browse files
jkbradleymengxr
authored andcommitted
[SPARK-5561] [MLLIB] Generalized PeriodicCheckpointer for RDDs and Graphs
PeriodicGraphCheckpointer was introduced for Latent Dirichlet Allocation (LDA), but it was meant to be generalized to work with Graphs, RDDs, and other data structures based on RDDs. This PR generalizes it. For those who are not familiar with the periodic checkpointer, it tries to automatically handle persisting/unpersisting and checkpointing/removing checkpoint files in a lineage of RDD-based objects. I need it generalized to use with GradientBoostedTrees [https://issues.apache.org/jira/browse/SPARK-6684]. It should be useful for other iterative algorithms as well. Changes I made: * Copied PeriodicGraphCheckpointer to PeriodicCheckpointer. * Within PeriodicCheckpointer, I created abstract methods for the basic operations (checkpoint, persist, etc.). * The subclasses for Graphs and RDDs implement those abstract methods. * I copied the test suite for the graph checkpointer and made tiny modifications to make it work for RDDs. To review this PR, I recommend doing 2 diffs: (1) diff between the old PeriodicGraphCheckpointer.scala and the new PeriodicCheckpointer.scala (2) diff between the 2 test suites CCing andrewor14 in case there are relevant changes to checkpointing. CCing feynmanliang in case you're interested in learning about checkpointing. CCing mengxr for final OK. Thanks all! Author: Joseph K. Bradley <[email protected]> Closes apache#7728 from jkbradley/gbt-checkpoint and squashes the following commits: d41902c [Joseph K. Bradley] Oops, forgot to update an extra time in the checkpointer tests, after the last commit. I'll fix that. I'll also make some of the checkpointer methods protected, which I should have done before. 32b23b8 [Joseph K. Bradley] fixed usage of checkpointer in lda 0b3dbc0 [Joseph K. Bradley] Changed checkpointer constructor not to take initial data. 568918c [Joseph K. Bradley] Generalized PeriodicGraphCheckpointer to PeriodicCheckpointer, with subclasses for RDDs and Graphs.
1 parent d31c618 commit c581593

File tree

6 files changed

+452
-99
lines changed

6 files changed

+452
-99
lines changed

mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -142,8 +142,8 @@ final class EMLDAOptimizer extends LDAOptimizer {
142142
this.k = k
143143
this.vocabSize = docs.take(1).head._2.size
144144
this.checkpointInterval = lda.getCheckpointInterval
145-
this.graphCheckpointer = new
146-
PeriodicGraphCheckpointer[TopicCounts, TokenCount](graph, checkpointInterval)
145+
this.graphCheckpointer = new PeriodicGraphCheckpointer[TopicCounts, TokenCount](
146+
checkpointInterval, graph.vertices.sparkContext)
147147
this.globalTopicTotals = computeGlobalTopicTotals()
148148
this
149149
}
@@ -188,7 +188,7 @@ final class EMLDAOptimizer extends LDAOptimizer {
188188
// Update the vertex descriptors with the new counts.
189189
val newGraph = GraphImpl.fromExistingRDDs(docTopicDistributions, graph.edges)
190190
graph = newGraph
191-
graphCheckpointer.updateGraph(newGraph)
191+
graphCheckpointer.update(newGraph)
192192
globalTopicTotals = computeGlobalTopicTotals()
193193
this
194194
}
Lines changed: 154 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,154 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.mllib.impl
19+
20+
import scala.collection.mutable
21+
22+
import org.apache.hadoop.fs.{Path, FileSystem}
23+
24+
import org.apache.spark.{SparkContext, Logging}
25+
import org.apache.spark.storage.StorageLevel
26+
27+
28+
/**
29+
* This abstraction helps with persisting and checkpointing RDDs and types derived from RDDs
30+
* (such as Graphs and DataFrames). In documentation, we use the phrase "Dataset" to refer to
31+
* the distributed data type (RDD, Graph, etc.).
32+
*
33+
* Specifically, this abstraction automatically handles persisting and (optionally) checkpointing,
34+
* as well as unpersisting and removing checkpoint files.
35+
*
36+
* Users should call update() when a new Dataset has been created,
37+
* before the Dataset has been materialized. After updating [[PeriodicCheckpointer]], users are
38+
* responsible for materializing the Dataset to ensure that persisting and checkpointing actually
39+
* occur.
40+
*
41+
* When update() is called, this does the following:
42+
* - Persist new Dataset (if not yet persisted), and put in queue of persisted Datasets.
43+
* - Unpersist Datasets from queue until there are at most 3 persisted Datasets.
44+
* - If using checkpointing and the checkpoint interval has been reached,
45+
* - Checkpoint the new Dataset, and put in a queue of checkpointed Datasets.
46+
* - Remove older checkpoints.
47+
*
48+
* WARNINGS:
49+
* - This class should NOT be copied (since copies may conflict on which Datasets should be
50+
* checkpointed).
51+
* - This class removes checkpoint files once later Datasets have been checkpointed.
52+
* However, references to the older Datasets will still return isCheckpointed = true.
53+
*
54+
* @param checkpointInterval Datasets will be checkpointed at this interval
55+
* @param sc SparkContext for the Datasets given to this checkpointer
56+
* @tparam T Dataset type, such as RDD[Double]
57+
*/
58+
private[mllib] abstract class PeriodicCheckpointer[T](
59+
val checkpointInterval: Int,
60+
val sc: SparkContext) extends Logging {
61+
62+
/** FIFO queue of past checkpointed Datasets */
63+
private val checkpointQueue = mutable.Queue[T]()
64+
65+
/** FIFO queue of past persisted Datasets */
66+
private val persistedQueue = mutable.Queue[T]()
67+
68+
/** Number of times [[update()]] has been called */
69+
private var updateCount = 0
70+
71+
/**
72+
* Update with a new Dataset. Handle persistence and checkpointing as needed.
73+
* Since this handles persistence and checkpointing, this should be called before the Dataset
74+
* has been materialized.
75+
*
76+
* @param newData New Dataset created from previous Datasets in the lineage.
77+
*/
78+
def update(newData: T): Unit = {
79+
persist(newData)
80+
persistedQueue.enqueue(newData)
81+
// We try to maintain 2 Datasets in persistedQueue to support the semantics of this class:
82+
// Users should call [[update()]] when a new Dataset has been created,
83+
// before the Dataset has been materialized.
84+
while (persistedQueue.size > 3) {
85+
val dataToUnpersist = persistedQueue.dequeue()
86+
unpersist(dataToUnpersist)
87+
}
88+
updateCount += 1
89+
90+
// Handle checkpointing (after persisting)
91+
if ((updateCount % checkpointInterval) == 0 && sc.getCheckpointDir.nonEmpty) {
92+
// Add new checkpoint before removing old checkpoints.
93+
checkpoint(newData)
94+
checkpointQueue.enqueue(newData)
95+
// Remove checkpoints before the latest one.
96+
var canDelete = true
97+
while (checkpointQueue.size > 1 && canDelete) {
98+
// Delete the oldest checkpoint only if the next checkpoint exists.
99+
if (isCheckpointed(checkpointQueue.head)) {
100+
removeCheckpointFile()
101+
} else {
102+
canDelete = false
103+
}
104+
}
105+
}
106+
}
107+
108+
/** Checkpoint the Dataset */
109+
protected def checkpoint(data: T): Unit
110+
111+
/** Return true iff the Dataset is checkpointed */
112+
protected def isCheckpointed(data: T): Boolean
113+
114+
/**
115+
* Persist the Dataset.
116+
* Note: This should handle checking the current [[StorageLevel]] of the Dataset.
117+
*/
118+
protected def persist(data: T): Unit
119+
120+
/** Unpersist the Dataset */
121+
protected def unpersist(data: T): Unit
122+
123+
/** Get list of checkpoint files for this given Dataset */
124+
protected def getCheckpointFiles(data: T): Iterable[String]
125+
126+
/**
127+
* Call this at the end to delete any remaining checkpoint files.
128+
*/
129+
def deleteAllCheckpoints(): Unit = {
130+
while (checkpointQueue.nonEmpty) {
131+
removeCheckpointFile()
132+
}
133+
}
134+
135+
/**
136+
* Dequeue the oldest checkpointed Dataset, and remove its checkpoint files.
137+
* This prints a warning but does not fail if the files cannot be removed.
138+
*/
139+
private def removeCheckpointFile(): Unit = {
140+
val old = checkpointQueue.dequeue()
141+
// Since the old checkpoint is not deleted by Spark, we manually delete it.
142+
val fs = FileSystem.get(sc.hadoopConfiguration)
143+
getCheckpointFiles(old).foreach { checkpointFile =>
144+
try {
145+
fs.delete(new Path(checkpointFile), true)
146+
} catch {
147+
case e: Exception =>
148+
logWarning("PeriodicCheckpointer could not remove old checkpoint file: " +
149+
checkpointFile)
150+
}
151+
}
152+
}
153+
154+
}

mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala

Lines changed: 16 additions & 89 deletions
Original file line numberDiff line numberDiff line change
@@ -17,11 +17,7 @@
1717

1818
package org.apache.spark.mllib.impl
1919

20-
import scala.collection.mutable
21-
22-
import org.apache.hadoop.fs.{Path, FileSystem}
23-
24-
import org.apache.spark.Logging
20+
import org.apache.spark.SparkContext
2521
import org.apache.spark.graphx.Graph
2622
import org.apache.spark.storage.StorageLevel
2723

@@ -31,12 +27,12 @@ import org.apache.spark.storage.StorageLevel
3127
* Specifically, it automatically handles persisting and (optionally) checkpointing, as well as
3228
* unpersisting and removing checkpoint files.
3329
*
34-
* Users should call [[PeriodicGraphCheckpointer.updateGraph()]] when a new graph has been created,
30+
* Users should call update() when a new graph has been created,
3531
* before the graph has been materialized. After updating [[PeriodicGraphCheckpointer]], users are
3632
* responsible for materializing the graph to ensure that persisting and checkpointing actually
3733
* occur.
3834
*
39-
* When [[PeriodicGraphCheckpointer.updateGraph()]] is called, this does the following:
35+
* When update() is called, this does the following:
4036
* - Persist new graph (if not yet persisted), and put in queue of persisted graphs.
4137
* - Unpersist graphs from queue until there are at most 3 persisted graphs.
4238
* - If using checkpointing and the checkpoint interval has been reached,
@@ -52,7 +48,7 @@ import org.apache.spark.storage.StorageLevel
5248
* Example usage:
5349
* {{{
5450
* val (graph1, graph2, graph3, ...) = ...
55-
* val cp = new PeriodicGraphCheckpointer(graph1, dir, 2)
51+
* val cp = new PeriodicGraphCheckpointer(2, sc)
5652
* graph1.vertices.count(); graph1.edges.count()
5753
* // persisted: graph1
5854
* cp.updateGraph(graph2)
@@ -73,99 +69,30 @@ import org.apache.spark.storage.StorageLevel
7369
* // checkpointed: graph4
7470
* }}}
7571
*
76-
* @param currentGraph Initial graph
7772
* @param checkpointInterval Graphs will be checkpointed at this interval
7873
* @tparam VD Vertex descriptor type
7974
* @tparam ED Edge descriptor type
8075
*
81-
* TODO: Generalize this for Graphs and RDDs, and move it out of MLlib.
76+
* TODO: Move this out of MLlib?
8277
*/
8378
private[mllib] class PeriodicGraphCheckpointer[VD, ED](
84-
var currentGraph: Graph[VD, ED],
85-
val checkpointInterval: Int) extends Logging {
86-
87-
/** FIFO queue of past checkpointed RDDs */
88-
private val checkpointQueue = mutable.Queue[Graph[VD, ED]]()
89-
90-
/** FIFO queue of past persisted RDDs */
91-
private val persistedQueue = mutable.Queue[Graph[VD, ED]]()
92-
93-
/** Number of times [[updateGraph()]] has been called */
94-
private var updateCount = 0
95-
96-
/**
97-
* Spark Context for the Graphs given to this checkpointer.
98-
* NOTE: This code assumes that only one SparkContext is used for the given graphs.
99-
*/
100-
private val sc = currentGraph.vertices.sparkContext
79+
checkpointInterval: Int,
80+
sc: SparkContext)
81+
extends PeriodicCheckpointer[Graph[VD, ED]](checkpointInterval, sc) {
10182

102-
updateGraph(currentGraph)
83+
override protected def checkpoint(data: Graph[VD, ED]): Unit = data.checkpoint()
10384

104-
/**
105-
* Update [[currentGraph]] with a new graph. Handle persistence and checkpointing as needed.
106-
* Since this handles persistence and checkpointing, this should be called before the graph
107-
* has been materialized.
108-
*
109-
* @param newGraph New graph created from previous graphs in the lineage.
110-
*/
111-
def updateGraph(newGraph: Graph[VD, ED]): Unit = {
112-
if (newGraph.vertices.getStorageLevel == StorageLevel.NONE) {
113-
newGraph.persist()
114-
}
115-
persistedQueue.enqueue(newGraph)
116-
// We try to maintain 2 Graphs in persistedQueue to support the semantics of this class:
117-
// Users should call [[updateGraph()]] when a new graph has been created,
118-
// before the graph has been materialized.
119-
while (persistedQueue.size > 3) {
120-
val graphToUnpersist = persistedQueue.dequeue()
121-
graphToUnpersist.unpersist(blocking = false)
122-
}
123-
updateCount += 1
85+
override protected def isCheckpointed(data: Graph[VD, ED]): Boolean = data.isCheckpointed
12486

125-
// Handle checkpointing (after persisting)
126-
if ((updateCount % checkpointInterval) == 0 && sc.getCheckpointDir.nonEmpty) {
127-
// Add new checkpoint before removing old checkpoints.
128-
newGraph.checkpoint()
129-
checkpointQueue.enqueue(newGraph)
130-
// Remove checkpoints before the latest one.
131-
var canDelete = true
132-
while (checkpointQueue.size > 1 && canDelete) {
133-
// Delete the oldest checkpoint only if the next checkpoint exists.
134-
if (checkpointQueue.get(1).get.isCheckpointed) {
135-
removeCheckpointFile()
136-
} else {
137-
canDelete = false
138-
}
139-
}
87+
override protected def persist(data: Graph[VD, ED]): Unit = {
88+
if (data.vertices.getStorageLevel == StorageLevel.NONE) {
89+
data.persist()
14090
}
14191
}
14292

143-
/**
144-
* Call this at the end to delete any remaining checkpoint files.
145-
*/
146-
def deleteAllCheckpoints(): Unit = {
147-
while (checkpointQueue.size > 0) {
148-
removeCheckpointFile()
149-
}
150-
}
93+
override protected def unpersist(data: Graph[VD, ED]): Unit = data.unpersist(blocking = false)
15194

152-
/**
153-
* Dequeue the oldest checkpointed Graph, and remove its checkpoint files.
154-
* This prints a warning but does not fail if the files cannot be removed.
155-
*/
156-
private def removeCheckpointFile(): Unit = {
157-
val old = checkpointQueue.dequeue()
158-
// Since the old checkpoint is not deleted by Spark, we manually delete it.
159-
val fs = FileSystem.get(sc.hadoopConfiguration)
160-
old.getCheckpointFiles.foreach { checkpointFile =>
161-
try {
162-
fs.delete(new Path(checkpointFile), true)
163-
} catch {
164-
case e: Exception =>
165-
logWarning("PeriodicGraphCheckpointer could not remove old checkpoint file: " +
166-
checkpointFile)
167-
}
168-
}
95+
override protected def getCheckpointFiles(data: Graph[VD, ED]): Iterable[String] = {
96+
data.getCheckpointFiles
16997
}
170-
17198
}

0 commit comments

Comments
 (0)