forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-5561] [MLLIB] Generalized PeriodicCheckpointer for RDDs and Gr…
…aphs 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.
- Loading branch information
Showing
6 changed files
with
452 additions
and
99 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
154 changes: 154 additions & 0 deletions
154
mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,154 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.mllib.impl | ||
|
||
import scala.collection.mutable | ||
|
||
import org.apache.hadoop.fs.{Path, FileSystem} | ||
|
||
import org.apache.spark.{SparkContext, Logging} | ||
import org.apache.spark.storage.StorageLevel | ||
|
||
|
||
/** | ||
* This abstraction helps with persisting and checkpointing RDDs and types derived from RDDs | ||
* (such as Graphs and DataFrames). In documentation, we use the phrase "Dataset" to refer to | ||
* the distributed data type (RDD, Graph, etc.). | ||
* | ||
* Specifically, this abstraction automatically handles persisting and (optionally) checkpointing, | ||
* as well as unpersisting and removing checkpoint files. | ||
* | ||
* Users should call update() when a new Dataset has been created, | ||
* before the Dataset has been materialized. After updating [[PeriodicCheckpointer]], users are | ||
* responsible for materializing the Dataset to ensure that persisting and checkpointing actually | ||
* occur. | ||
* | ||
* When update() is called, this does the following: | ||
* - Persist new Dataset (if not yet persisted), and put in queue of persisted Datasets. | ||
* - Unpersist Datasets from queue until there are at most 3 persisted Datasets. | ||
* - If using checkpointing and the checkpoint interval has been reached, | ||
* - Checkpoint the new Dataset, and put in a queue of checkpointed Datasets. | ||
* - Remove older checkpoints. | ||
* | ||
* WARNINGS: | ||
* - This class should NOT be copied (since copies may conflict on which Datasets should be | ||
* checkpointed). | ||
* - This class removes checkpoint files once later Datasets have been checkpointed. | ||
* However, references to the older Datasets will still return isCheckpointed = true. | ||
* | ||
* @param checkpointInterval Datasets will be checkpointed at this interval | ||
* @param sc SparkContext for the Datasets given to this checkpointer | ||
* @tparam T Dataset type, such as RDD[Double] | ||
*/ | ||
private[mllib] abstract class PeriodicCheckpointer[T]( | ||
val checkpointInterval: Int, | ||
val sc: SparkContext) extends Logging { | ||
|
||
/** FIFO queue of past checkpointed Datasets */ | ||
private val checkpointQueue = mutable.Queue[T]() | ||
|
||
/** FIFO queue of past persisted Datasets */ | ||
private val persistedQueue = mutable.Queue[T]() | ||
|
||
/** Number of times [[update()]] has been called */ | ||
private var updateCount = 0 | ||
|
||
/** | ||
* Update with a new Dataset. Handle persistence and checkpointing as needed. | ||
* Since this handles persistence and checkpointing, this should be called before the Dataset | ||
* has been materialized. | ||
* | ||
* @param newData New Dataset created from previous Datasets in the lineage. | ||
*/ | ||
def update(newData: T): Unit = { | ||
persist(newData) | ||
persistedQueue.enqueue(newData) | ||
// We try to maintain 2 Datasets in persistedQueue to support the semantics of this class: | ||
// Users should call [[update()]] when a new Dataset has been created, | ||
// before the Dataset has been materialized. | ||
while (persistedQueue.size > 3) { | ||
val dataToUnpersist = persistedQueue.dequeue() | ||
unpersist(dataToUnpersist) | ||
} | ||
updateCount += 1 | ||
|
||
// Handle checkpointing (after persisting) | ||
if ((updateCount % checkpointInterval) == 0 && sc.getCheckpointDir.nonEmpty) { | ||
// Add new checkpoint before removing old checkpoints. | ||
checkpoint(newData) | ||
checkpointQueue.enqueue(newData) | ||
// Remove checkpoints before the latest one. | ||
var canDelete = true | ||
while (checkpointQueue.size > 1 && canDelete) { | ||
// Delete the oldest checkpoint only if the next checkpoint exists. | ||
if (isCheckpointed(checkpointQueue.head)) { | ||
removeCheckpointFile() | ||
} else { | ||
canDelete = false | ||
} | ||
} | ||
} | ||
} | ||
|
||
/** Checkpoint the Dataset */ | ||
protected def checkpoint(data: T): Unit | ||
|
||
/** Return true iff the Dataset is checkpointed */ | ||
protected def isCheckpointed(data: T): Boolean | ||
|
||
/** | ||
* Persist the Dataset. | ||
* Note: This should handle checking the current [[StorageLevel]] of the Dataset. | ||
*/ | ||
protected def persist(data: T): Unit | ||
|
||
/** Unpersist the Dataset */ | ||
protected def unpersist(data: T): Unit | ||
|
||
/** Get list of checkpoint files for this given Dataset */ | ||
protected def getCheckpointFiles(data: T): Iterable[String] | ||
|
||
/** | ||
* Call this at the end to delete any remaining checkpoint files. | ||
*/ | ||
def deleteAllCheckpoints(): Unit = { | ||
while (checkpointQueue.nonEmpty) { | ||
removeCheckpointFile() | ||
} | ||
} | ||
|
||
/** | ||
* Dequeue the oldest checkpointed Dataset, and remove its checkpoint files. | ||
* This prints a warning but does not fail if the files cannot be removed. | ||
*/ | ||
private def removeCheckpointFile(): Unit = { | ||
val old = checkpointQueue.dequeue() | ||
// Since the old checkpoint is not deleted by Spark, we manually delete it. | ||
val fs = FileSystem.get(sc.hadoopConfiguration) | ||
getCheckpointFiles(old).foreach { checkpointFile => | ||
try { | ||
fs.delete(new Path(checkpointFile), true) | ||
} catch { | ||
case e: Exception => | ||
logWarning("PeriodicCheckpointer could not remove old checkpoint file: " + | ||
checkpointFile) | ||
} | ||
} | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.