aboutsummaryrefslogtreecommitdiff
path: root/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala
blob: 391f89aa1489f055ab5a9d9888731225d1388e6c (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
/*
 * 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.{FileSystem, Path}

import org.apache.spark.SparkContext
import org.apache.spark.internal.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)
      }
    }
  }

}