Github user zsxwing commented on a diff in the pull request: https://github.com/apache/spark/pull/9428#discussion_r44344621 --- Diff: core/src/main/scala/org/apache/spark/util/CheckpointingIterator.scala --- @@ -0,0 +1,225 @@ +/* + * 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.util + +import java.io.IOException +import java.io.OutputStream +import java.util.concurrent.ConcurrentHashMap + +import scala.reflect.ClassTag +import scala.util.control.NonFatal + +import org.apache.hadoop.fs.{FileSystem, Path} + +import org.apache.spark._ +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.{RDD, ReliableCheckpointRDD} +import org.apache.spark.storage.RDDBlockId + +/** + * Wrapper around an iterator which writes checkpoint data to HDFS while running action on + * an RDD. + * + * @param id the unique id for a partition of an RDD + * @param values the data to be checkpointed + * @param fs the FileSystem to use + * @param tempOutputPath the temp path to write the checkpoint data + * @param finalOutputPath the final path to move the temp file to when finishing checkpointing + * @param context the task context + * @param blockSize the block size for writing the checkpoint data + */ +private[spark] class CheckpointingIterator[T: ClassTag]( + id: RDDBlockId, + values: Iterator[T], + fs: FileSystem, + tempOutputPath: Path, + finalOutputPath: Path, + context: TaskContext, + blockSize: Int) extends Iterator[T] with Logging { + + private[this] var completed = false + + // We don't know if the task is successful. So it's possible that we still checkpoint the + // remaining values even if the task is failed. + // TODO optimize the failure case if we can know the task status + context.addTaskCompletionListener { _ => complete() } + + private[this] val fileOutputStream: OutputStream = { + val bufferSize = SparkEnv.get.conf.getInt("spark.buffer.size", 65536) + if (blockSize < 0) { + fs.create(tempOutputPath, false, bufferSize) + } else { + // This is mainly for testing purpose + fs.create(tempOutputPath, false, bufferSize, fs.getDefaultReplication, blockSize) + } + } + + private[this] val serializeStream = + SparkEnv.get.serializer.newInstance().serializeStream(fileOutputStream) + + /** + * Called when this iterator is on the last element by `hasNext`. + * This method will rename temporary output path to final output path of checkpoint data. + */ + private[this] def complete(): Unit = { + if (completed) { + return + } + + if (serializeStream == null) { + // There is some exception when creating serializeStream, we only need to clean up the + // resources. + cleanup() + return + } + + while (values.hasNext) { + serializeStream.writeObject(values.next) + } + + completed = true + CheckpointingIterator.releaseLockForPartition(id) + serializeStream.close() + + if (!fs.rename(tempOutputPath, finalOutputPath)) { + if (!fs.exists(finalOutputPath)) { + logInfo("Deleting tempOutputPath " + tempOutputPath) + fs.delete(tempOutputPath, false) + throw new IOException("Checkpoint failed: failed to save output of task: " + + context.attemptNumber + " and final output path does not exist") + } else { + // Some other copy of this task must've finished before us and renamed it + logInfo("Final output path " + finalOutputPath + " already exists; not overwriting it") + fs.delete(tempOutputPath, false) + } + } + } + + private[this] def cleanup(): Unit = { + completed = true + CheckpointingIterator.releaseLockForPartition(id) + if (serializeStream != null) { + serializeStream.close() + } + fs.delete(tempOutputPath, false) + } + + private[this] def cleanupOnFailure[A](body: => A): A = { + try { + body + } catch { + case e: Throwable => --- End diff -- Use `Throwable` here because it will be thrown later. It's better to cleanup as well for fatal errors.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org