|
| 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 | + * Example usage: |
| 55 | + * {{{ |
| 56 | + * val (data1, data2, data3, ...) = ... |
| 57 | + * val cp = new PeriodicCheckpointer(data1, dir, 2) |
| 58 | + * data1.count(); |
| 59 | + * // persisted: data1 |
| 60 | + * cp.update(data2) |
| 61 | + * data2.count(); |
| 62 | + * // persisted: data1, data2 |
| 63 | + * // checkpointed: data2 |
| 64 | + * cp.update(data3) |
| 65 | + * data3.count(); |
| 66 | + * // persisted: data1, data2, data3 |
| 67 | + * // checkpointed: data2 |
| 68 | + * cp.update(data4) |
| 69 | + * data4.count(); |
| 70 | + * // persisted: data2, data3, data4 |
| 71 | + * // checkpointed: data4 |
| 72 | + * cp.update(data5) |
| 73 | + * data5.count(); |
| 74 | + * // persisted: data3, data4, data5 |
| 75 | + * // checkpointed: data4 |
| 76 | + * }}} |
| 77 | + * |
| 78 | + * @param currentData Initial Dataset |
| 79 | + * @param checkpointInterval Datasets will be checkpointed at this interval |
| 80 | + * @param sc SparkContext for the Datasets given to this checkpointer |
| 81 | + * @tparam T Dataset type, such as RDD[Double] |
| 82 | + */ |
| 83 | +private[mllib] abstract class PeriodicCheckpointer[T]( |
| 84 | + var currentData: T, |
| 85 | + val checkpointInterval: Int, |
| 86 | + val sc: SparkContext) extends Logging { |
| 87 | + |
| 88 | + /** FIFO queue of past checkpointed Datasets */ |
| 89 | + private val checkpointQueue = mutable.Queue[T]() |
| 90 | + |
| 91 | + /** FIFO queue of past persisted Datasets */ |
| 92 | + private val persistedQueue = mutable.Queue[T]() |
| 93 | + |
| 94 | + /** Number of times [[update()]] has been called */ |
| 95 | + private var updateCount = 0 |
| 96 | + |
| 97 | + update(currentData) |
| 98 | + |
| 99 | + /** |
| 100 | + * Update [[currentData]] with a new Dataset. Handle persistence and checkpointing as needed. |
| 101 | + * Since this handles persistence and checkpointing, this should be called before the Dataset |
| 102 | + * has been materialized. |
| 103 | + * |
| 104 | + * @param newData New Dataset created from previous Datasets in the lineage. |
| 105 | + */ |
| 106 | + def update(newData: T): Unit = { |
| 107 | + persist(newData) |
| 108 | + persistedQueue.enqueue(newData) |
| 109 | + // We try to maintain 2 Datasets in persistedQueue to support the semantics of this class: |
| 110 | + // Users should call [[update()]] when a new Dataset has been created, |
| 111 | + // before the Dataset has been materialized. |
| 112 | + while (persistedQueue.size > 3) { |
| 113 | + val dataToUnpersist = persistedQueue.dequeue() |
| 114 | + unpersist(dataToUnpersist) |
| 115 | + } |
| 116 | + updateCount += 1 |
| 117 | + |
| 118 | + // Handle checkpointing (after persisting) |
| 119 | + if ((updateCount % checkpointInterval) == 0 && sc.getCheckpointDir.nonEmpty) { |
| 120 | + // Add new checkpoint before removing old checkpoints. |
| 121 | + checkpoint(newData) |
| 122 | + checkpointQueue.enqueue(newData) |
| 123 | + // Remove checkpoints before the latest one. |
| 124 | + var canDelete = true |
| 125 | + while (checkpointQueue.size > 1 && canDelete) { |
| 126 | + // Delete the oldest checkpoint only if the next checkpoint exists. |
| 127 | + if (isCheckpointed(checkpointQueue.get(1).get)) { |
| 128 | + removeCheckpointFile() |
| 129 | + } else { |
| 130 | + canDelete = false |
| 131 | + } |
| 132 | + } |
| 133 | + } |
| 134 | + |
| 135 | + currentData = newData |
| 136 | + } |
| 137 | + |
| 138 | + /** Checkpoint the Dataset */ |
| 139 | + def checkpoint(data: T): Unit |
| 140 | + |
| 141 | + /** Return true iff the Dataset is checkpointed */ |
| 142 | + def isCheckpointed(data: T): Boolean |
| 143 | + |
| 144 | + /** |
| 145 | + * Persist the Dataset. |
| 146 | + * Note: This should handle checking the current [[StorageLevel]] of the Dataset. |
| 147 | + */ |
| 148 | + def persist(data: T): Unit |
| 149 | + |
| 150 | + /** Unpersist the Dataset */ |
| 151 | + def unpersist(data: T): Unit |
| 152 | + |
| 153 | + /** Get list of checkpoint files for this given Dataset */ |
| 154 | + def getCheckpointFiles(data: T): Iterable[String] |
| 155 | + |
| 156 | + /** |
| 157 | + * Call this at the end to delete any remaining checkpoint files. |
| 158 | + */ |
| 159 | + def deleteAllCheckpoints(): Unit = { |
| 160 | + while (checkpointQueue.nonEmpty) { |
| 161 | + removeCheckpointFile() |
| 162 | + } |
| 163 | + } |
| 164 | + |
| 165 | + /** |
| 166 | + * Dequeue the oldest checkpointed Dataset, and remove its checkpoint files. |
| 167 | + * This prints a warning but does not fail if the files cannot be removed. |
| 168 | + */ |
| 169 | + private def removeCheckpointFile(): Unit = { |
| 170 | + val old = checkpointQueue.dequeue() |
| 171 | + // Since the old checkpoint is not deleted by Spark, we manually delete it. |
| 172 | + val fs = FileSystem.get(sc.hadoopConfiguration) |
| 173 | + getCheckpointFiles(old).foreach { checkpointFile => |
| 174 | + try { |
| 175 | + fs.delete(new Path(checkpointFile), true) |
| 176 | + } catch { |
| 177 | + case e: Exception => |
| 178 | + logWarning("PeriodicCheckpointer could not remove old checkpoint file: " + |
| 179 | + checkpointFile) |
| 180 | + } |
| 181 | + } |
| 182 | + } |
| 183 | + |
| 184 | +} |
0 commit comments