前言
在spark應用程序中,常常會遇到運算量很大經過很復雜的 Transformation才能得到的RDD即Lineage鏈較長、寬依賴的RDD,此時我們可以考慮將這個RDD持久化。
cache也是可以持久化到磁盤,只不過是直接將partition的輸出數據寫到磁盤,而checkpoint是在邏輯job完成后,若有需要checkpoint的RDD,再單獨啟動一個job去完成checkpoint,這樣該RDD就被計算了兩次,所以建議在有checkpoint的時候先將該RDD cache到內存,到時候直接寫到磁盤就行了。
checkpoint的實現
需要使用checkpoint都需要通過sparkcontext的setCheckpointDir方法設置一個目錄以存checkpoint的各種信息數據,下面我們來看看該方法:
def setCheckpointDir(directory: String) {
if (!isLocal && Utils.nonLocalPaths(directory).isEmpty) {
logWarning("Spark is not running in local mode, therefore the checkpoint directory " +
s"must not be on the local filesystem. Directory '$directory' " +
"appears to be on the local filesystem.")
}
checkpointDir = Option(directory).map { dir =>
val path = new Path(dir, UUID.randomUUID().toString)
val fs = path.getFileSystem(hadoopConfiguration)
fs.mkdirs(path)
fs.getFileStatus(path).getPath.toString
}
}
在非local模式下,directory必須是HDFS的目錄;在該目錄下創建一個以UUID生成的一個唯一的目錄名的目錄。
通過rdd.checkpoint()即可checkpoint此RDD
def checkpoint(): Unit = RDDCheckpointData.synchronized {
if (context.checkpointDir.isEmpty) {
throw new SparkException("Checkpoint directory has not been set in the SparkContext")
} else if (checkpointData.isEmpty) {
checkpointData = Some(new ReliableRDDCheckpointData(this))
}
}
先判斷是否設置了checkpointDir,再判斷checkpointData.isEmpty是否成立,checkpointData的定義是這樣的:
private[spark] var checkpointData: Option[RDDCheckpointData[T]] = None
RDDCheckpointData和RDD一一對應,保存著和checkpoint相關的信息。這里通過new ReliableRDDCheckpointData(this)實例化了checkpointData ,ReliableRDDCheckpointData是其子類,這里相當于是checkpoint的一個標記,并沒有真正執行checkpoint。
什么時候checkpoint
在有action動作時,會觸發sparkcontext對runJob的調用:
def runJob[T, U: ClassTag](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
resultHandler: (Int, U) => Unit): Unit = {
if (stopped.get()) {
throw new IllegalStateException("SparkContext has been shutdown")
}
val callSite = getCallSite
val cleanedFunc = clean(func)
logInfo("Starting job: " + callSite.shortForm)
if (conf.getBoolean("spark.logLineage", false)) {
logInfo("RDD's recursive dependencies:\n" + rdd.toDebugString)
}
dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, resultHandler, localProperties.get)
progressBar.foreach(_.finishAll())
rdd.doCheckpoint()
}
我們可以看到在執行完job后會執行 rdd.doCheckpoint(),這里就是對前面標記了的RDD的checkpoint,我們繼續看這個方法:
private[spark] def doCheckpoint(): Unit = {
RDDOperationScope.withScope(sc, "checkpoint", allowNesting = false, ignoreParent = true) {
if (!doCheckpointCalled) {
doCheckpointCalled = true
if (checkpointData.isDefined) {
if (checkpointAllMarkedAncestors) {
dependencies.foreach(_.rdd.doCheckpoint())
}
checkpointData.get.checkpoint()
} else {
dependencies.foreach(_.rdd.doCheckpoint())
}
}
}
}
先判斷是否已經被處理過checkpoint,沒有才執行,并將doCheckpointCalled 設為true,因為前面已經初始化過了checkpointData,所以checkpointData.isDefined也滿足,若想要把checkpointData定義過的RDD的parents也進行checkpoint的話,那么我們需要先對parents checkpoint。因為,如果RDD把自己checkpoint了,那么它就將lineage中它的parents給切除了。繼續跟進checkpointData.get.checkpoint()
final def checkpoint(): Unit = {
// Guard against multiple threads checkpointing the same RDD by
// atomically flipping the state of this RDDCheckpointData
RDDCheckpointData.synchronized {
if (cpState == Initialized) {
cpState = CheckpointingInProgress
} else {
return
}
}
val newRDD = doCheckpoint()
// Update our state and truncate the RDD lineage
RDDCheckpointData.synchronized {
cpRDD = Some(newRDD)
cpState = Checkpointed
rdd.markCheckpointed()
}
}
先將checkpoint的狀態改為CheckpointingInProgress,再執行doCheckpoint,返回一個newRDD,看doCheckpoint做了什么:
protected override def doCheckpoint(): CheckpointRDD[T] = {
val newRDD = ReliableCheckpointRDD.writeRDDToCheckpointDirectory(rdd, cpDir)
if (rdd.conf.getBoolean("spark.cleaner.referenceTracking.cleanCheckpoints", false)) {
rdd.context.cleaner.foreach { cleaner =>
cleaner.registerRDDCheckpointDataForCleanup(newRDD, rdd.id)
}
}
logInfo(s"Done checkpointing RDD ${rdd.id} to $cpDir, new parent is RDD ${newRDD.id}")
newRDD
}
ReliableCheckpointRDD.writeRDDToCheckpointDirectory(rdd, cpDir),將一個RDD寫入到多個checkpoint文件,并返回一個ReliableCheckpointRDD來代表這個RDD
def writeRDDToCheckpointDirectory[T: ClassTag](
originalRDD: RDD[T],
checkpointDir: String,
blockSize: Int = -1): ReliableCheckpointRDD[T] = {
val sc = originalRDD.sparkContext
// Create the output path for the checkpoint
val checkpointDirPath = new Path(checkpointDir)
val fs = checkpointDirPath.getFileSystem(sc.hadoopConfiguration)
if (!fs.mkdirs(checkpointDirPath)) {
throw new SparkException(s"Failed to create checkpoint path $checkpointDirPath")
}
// Save to file, and reload it as an RDD
val broadcastedConf = sc.broadcast(
new SerializableConfiguration(sc.hadoopConfiguration))
// TODO: This is expensive because it computes the RDD again unnecessarily (SPARK-8582)
sc.runJob(originalRDD,
writePartitionToCheckpointFile[T](checkpointDirPath.toString, broadcastedConf) _)
if (originalRDD.partitioner.nonEmpty) {
writePartitionerToCheckpointDir(sc, originalRDD.partitioner.get, checkpointDirPath)
}
val newRDD = new ReliableCheckpointRDD[T](
sc, checkpointDirPath.toString, originalRDD.partitioner)
if (newRDD.partitions.length != originalRDD.partitions.length) {
throw new SparkException(
s"Checkpoint RDD $newRDD(${newRDD.partitions.length}) has different " +
s"number of partitions from original RDD $originalRDD(${originalRDD.partitions.length})")
}
newRDD
}
獲取一些配置信息廣播輸出等操作,然后啟動一個Job去寫Checkpint文件,主要由ReliableCheckpointRDD.writeCheckpointFile來實現寫操作,寫完checkpoint后new一個ReliableCheckpointRDD實例返回,看看具體的writePartitionToCheckpointFile實現:
def writePartitionToCheckpointFile[T: ClassTag](
path: String,
broadcastedConf: Broadcast[SerializableConfiguration],
blockSize: Int = -1)(ctx: TaskContext, iterator: Iterator[T]) {
val env = SparkEnv.get
val outputDir = new Path(path)
val fs = outputDir.getFileSystem(broadcastedConf.value.value)
val finalOutputName = ReliableCheckpointRDD.checkpointFileName(ctx.partitionId())
val finalOutputPath = new Path(outputDir, finalOutputName)
val tempOutputPath =
new Path(outputDir, s".$finalOutputName-attempt-${ctx.attemptNumber()}")
if (fs.exists(tempOutputPath)) {
throw new IOException(s"Checkpoint failed: temporary path $tempOutputPath already exists")
}
val bufferSize = env.conf.getInt("spark.buffer.size", 65536)
val fileOutputStream = if (blockSize < 0) {
fs.create(tempOutputPath, false, bufferSize)
} else {
// This is mainly for testing purpose
fs.create(tempOutputPath, false, bufferSize,
fs.getDefaultReplication(fs.getWorkingDirectory), blockSize)
}
val serializer = env.serializer.newInstance()
val serializeStream = serializer.serializeStream(fileOutputStream)
Utils.tryWithSafeFinally {
serializeStream.writeAll(iterator)
} {
serializeStream.close()
}
if (!fs.rename(tempOutputPath, finalOutputPath)) {
if (!fs.exists(finalOutputPath)) {
logInfo(s"Deleting tempOutputPath $tempOutputPath")
fs.delete(tempOutputPath, false)
throw new IOException("Checkpoint failed: failed to save output of task: " +
s"${ctx.attemptNumber()} and final output path does not exist: $finalOutputPath")
} else {
// Some other copy of this task must've finished before us and renamed it
logInfo(s"Final output path $finalOutputPath already exists; not overwriting it")
if (!fs.delete(tempOutputPath, false)) {
logWarning(s"Error deleting ${tempOutputPath}")
}
}
}
}
這里的代碼就是普通的對HDFS寫文件的操作,將一個RDD partition的數據寫到checkpoint目錄下。
doCheckpoint()操作已經完成,返回了一個new RDD:ReliableCheckpointRDD引用給cpRDD,接著標記checkpoint的狀態為Checkpointed,rdd.markCheckpointed()干了什么呢?
private[spark] def markCheckpointed(): Unit = {
clearDependencies()
partitions_ = null
deps = null // Forget the constructor argument for dependencies too
}
最后再清除RDD的所有依賴。
寫checkpoint總結
- Initialized
- marked for checkpointing
- checkpointing in progress
- checkpointed
什么時候讀checkpoint
在需要讀取一個partition的數據時,會通過rdd.iterator() 去計算該 rdd 的 partition 的,我們來看RDD的iterator()實現:
final def iterator(split: Partition, context: TaskContext): Iterator[T] = {
if (storageLevel != StorageLevel.NONE) {
getOrCompute(split, context)
} else {
computeOrReadCheckpoint(split, context)
}
}
在cache中沒有讀到數據時再判斷該RDD是否被checkpoint過,isCheckpointedAndMaterialized就是在checkpoint成功時的一個狀態標記:cpState = Checkpointed。
private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] =
{
if (isCheckpointedAndMaterialized) {
firstParent[T].iterator(split, context)
} else {
compute(split, context)
}
}
當該RDD被成功checkpoint了,直接使用parent rdd 的 iterator() 也就是 CheckpointRDD.iterator(),否則直接調用該RDD的compute方法。
final def dependencies: Seq[Dependency[_]] = {
checkpointRDD.map(r => List(new OneToOneDependency(r))).getOrElse {
if (dependencies_ == null) {
dependencies_ = getDependencies
}
dependencies_
}
}
獲取RDD的依賴時,會先嘗試從checkpointRDD中獲取依賴,若成功則返回被OneToOneDependency包裝過的ReliableCheckpointRDD對象,否則獲取真正的依賴。