-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-1103] [WIP] Automatic garbage collection of RDD, shuffle and broadcast data #126
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 20 commits
1e752f1
80dd977
e427a9e
8512612
a24fefc
cb0a5a6
ae9da88
e61daa0
a7260d3
892b952
e1fba5f
f2881fd
620eca3
a007307
d2f8b97
6c9dcf6
c7ccef1
ba52e00
d0edef3
544ac86
e95479c
f201a8d
c92e4d9
0d17060
34f436f
fbfeec8
88904a3
e442246
8557c12
7edbc98
634a097
7ed72fb
5016375
f0aabb1
762a4d8
a6460d4
c5b1d98
a2cc8bc
ada45f0
cd72d19
b27f8e8
a430f06
104a89a
6222697
41c9ece
2b95b5e
4d05314
cff023c
d25a86e
f489fdc
61b8d6e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,105 +21,106 @@ import java.lang.ref.{ReferenceQueue, WeakReference} | |
|
|
||
| import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} | ||
|
|
||
| import org.apache.spark.broadcast.Broadcast | ||
| import org.apache.spark.rdd.RDD | ||
|
|
||
| /** Listener class used for testing when any item has been cleaned by the Cleaner class */ | ||
| private[spark] trait CleanerListener { | ||
| def rddCleaned(rddId: Int) | ||
| def shuffleCleaned(shuffleId: Int) | ||
| } | ||
| /** | ||
| * Classes that represent cleaning tasks. | ||
| */ | ||
| private sealed trait CleanupTask | ||
| private case class CleanRDD(rddId: Int) extends CleanupTask | ||
| private case class CleanShuffle(shuffleId: Int) extends CleanupTask | ||
| private case class CleanBroadcast(broadcastId: Long) extends CleanupTask | ||
|
|
||
| /** | ||
| * Cleans RDDs and shuffle data. | ||
| * A WeakReference associated with a CleanupTask. | ||
| * | ||
| * When the referent object becomes only weakly reachable, the corresponding | ||
| * CleanupTaskWeakReference is automatically added to the given reference queue. | ||
| */ | ||
| private class CleanupTaskWeakReference( | ||
| val task: CleanupTask, | ||
| referent: AnyRef, | ||
| referenceQueue: ReferenceQueue[AnyRef]) | ||
| extends WeakReference(referent, referenceQueue) | ||
|
|
||
| /** | ||
| * An asynchronous cleaner for RDD, shuffle, and broadcast state. | ||
| * | ||
| * This maintains a weak reference for each RDD, ShuffleDependency, and Broadcast of interest, | ||
| * to be processed when the associated object goes out of scope of the application. Actual | ||
| * cleanup is performed in a separate daemon thread. | ||
| */ | ||
| private[spark] class ContextCleaner(sc: SparkContext) extends Logging { | ||
|
|
||
| /** Classes to represent cleaning tasks */ | ||
| private sealed trait CleanupTask | ||
| private case class CleanRDD(rddId: Int) extends CleanupTask | ||
| private case class CleanShuffle(shuffleId: Int) extends CleanupTask | ||
| // TODO: add CleanBroadcast | ||
| private val referenceBuffer = new ArrayBuffer[CleanupTaskWeakReference] | ||
| with SynchronizedBuffer[CleanupTaskWeakReference] | ||
|
|
||
| private val referenceBuffer = new ArrayBuffer[WeakReferenceWithCleanupTask] | ||
| with SynchronizedBuffer[WeakReferenceWithCleanupTask] | ||
| private val referenceQueue = new ReferenceQueue[AnyRef] | ||
|
|
||
| private val listeners = new ArrayBuffer[CleanerListener] | ||
| with SynchronizedBuffer[CleanerListener] | ||
|
|
||
| private val cleaningThread = new Thread() { override def run() { keepCleaning() }} | ||
|
|
||
| private val REF_QUEUE_POLL_TIMEOUT = 100 | ||
|
|
||
| @volatile private var stopped = false | ||
|
|
||
| private class WeakReferenceWithCleanupTask(referent: AnyRef, val task: CleanupTask) | ||
| extends WeakReference(referent, referenceQueue) | ||
| /** Attach a listener object to get information of when objects are cleaned. */ | ||
| def attachListener(listener: CleanerListener) { | ||
| listeners += listener | ||
| } | ||
|
|
||
| /** Start the cleaner */ | ||
| /** Start the cleaner. */ | ||
| def start() { | ||
| cleaningThread.setDaemon(true) | ||
| cleaningThread.setName("ContextCleaner") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'd say "Spark ContextCleaner" here since Spark programs have tons of threads from different libraries (akka, jetty, etc). |
||
| cleaningThread.start() | ||
| } | ||
|
|
||
| /** Stop the cleaner */ | ||
| /** Stop the cleaner. */ | ||
| def stop() { | ||
| stopped = true | ||
| cleaningThread.interrupt() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Isn't there a risk that calling
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ya it looks like a lot of the clean-up implementations interact with shared state like the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are you suggesting that we just let the thread terminate automatically with the JVM?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No I'm suggesting that you rely on the existing |
||
| } | ||
|
|
||
| /** | ||
| * Register a RDD for cleanup when it is garbage collected. | ||
| */ | ||
| /** Register a RDD for cleanup when it is garbage collected. */ | ||
| def registerRDDForCleanup(rdd: RDD[_]) { | ||
| registerForCleanup(rdd, CleanRDD(rdd.id)) | ||
| } | ||
|
|
||
| /** | ||
| * Register a shuffle dependency for cleanup when it is garbage collected. | ||
| */ | ||
| /** Register a ShuffleDependency for cleanup when it is garbage collected. */ | ||
| def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _]) { | ||
| registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) | ||
| } | ||
|
|
||
| /** Cleanup RDD. */ | ||
| def cleanupRDD(rdd: RDD[_]) { | ||
| doCleanupRDD(rdd.id) | ||
| } | ||
|
|
||
| /** Cleanup shuffle. */ | ||
| def cleanupShuffle(shuffleDependency: ShuffleDependency[_, _]) { | ||
| doCleanupShuffle(shuffleDependency.shuffleId) | ||
| } | ||
|
|
||
| /** Attach a listener object to get information of when objects are cleaned. */ | ||
| def attachListener(listener: CleanerListener) { | ||
| listeners += listener | ||
| /** Register a Broadcast for cleanup when it is garbage collected. */ | ||
| def registerBroadcastForCleanup[T](broadcast: Broadcast[T]) { | ||
| registerForCleanup(broadcast, CleanBroadcast(broadcast.id)) | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Once you rebase against master, sc.unpersist pretty much does exactly the same thing. (You might have to change the argument from an RDD to an Int though). |
||
|
|
||
| /** Register an object for cleanup. */ | ||
| private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask) { | ||
| referenceBuffer += new WeakReferenceWithCleanupTask(objectForCleanup, task) | ||
| referenceBuffer += new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue) | ||
| } | ||
|
|
||
| /** Keep cleaning RDDs and shuffle data */ | ||
| /** Keep cleaning RDD, shuffle, and broadcast state. */ | ||
| private def keepCleaning() { | ||
| while (!isStopped) { | ||
| while (!stopped) { | ||
| try { | ||
| val reference = Option(referenceQueue.remove(REF_QUEUE_POLL_TIMEOUT)) | ||
| .map(_.asInstanceOf[WeakReferenceWithCleanupTask]) | ||
| val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) | ||
| .map(_.asInstanceOf[CleanupTaskWeakReference]) | ||
| reference.map(_.task).foreach { task => | ||
| logDebug("Got cleaning task " + task) | ||
| referenceBuffer -= reference.get | ||
| task match { | ||
| case CleanRDD(rddId) => doCleanupRDD(rddId) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. From what I can tell here Otherwise if there is, e.g. GC on one of the slaves, or one of the shuffles takes a long time to delete all of the files, then you could end up in a situation where you are sitting in this loop for a long time. That doesn't seem super desirable.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah I see - actually maybe the other ones are non blocking as well. It would be good to say in, e.g. the |
||
| case CleanShuffle(shuffleId) => doCleanupShuffle(shuffleId) | ||
| case CleanBroadcast(broadcastId) => doCleanupBroadcast(broadcastId) | ||
| } | ||
| } | ||
| } catch { | ||
| case ie: InterruptedException => | ||
| if (!isStopped) logWarning("Cleaning thread interrupted") | ||
| if (!stopped) logWarning("Cleaning thread interrupted") | ||
| case t: Throwable => logError("Error in cleaning thread", t) | ||
| } | ||
| } | ||
|
|
@@ -129,7 +130,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { | |
| private def doCleanupRDD(rddId: Int) { | ||
| try { | ||
| logDebug("Cleaning RDD " + rddId) | ||
| sc.unpersistRDD(rddId, false) | ||
| sc.unpersistRDD(rddId, blocking = false) | ||
| listeners.foreach(_.rddCleaned(rddId)) | ||
| logInfo("Cleaned RDD " + rddId) | ||
| } catch { | ||
|
|
@@ -150,10 +151,46 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { | |
| } | ||
| } | ||
|
|
||
| private def mapOutputTrackerMaster = | ||
| sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] | ||
| /** Perform broadcast cleanup. */ | ||
| private def doCleanupBroadcast(broadcastId: Long) { | ||
| try { | ||
| logDebug("Cleaning broadcast " + broadcastId) | ||
| broadcastManager.unbroadcast(broadcastId, removeFromDriver = true) | ||
| listeners.foreach(_.broadcastCleaned(broadcastId)) | ||
| logInfo("Cleaned broadcast " + broadcastId) | ||
| } catch { | ||
| case t: Throwable => logError("Error cleaning broadcast " + broadcastId, t) | ||
| } | ||
| } | ||
|
|
||
| private def blockManagerMaster = sc.env.blockManager.master | ||
| private def broadcastManager = sc.env.broadcastManager | ||
| private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] | ||
|
|
||
| // Used for testing | ||
|
|
||
| def cleanupRDD(rdd: RDD[_]) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't get why these need to be exposed (each of these is only called once). Why not just call
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Fixed. Dont really need private[spark] as the whole class is private[spark]. So just making them public is sufficient.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will push with all updates. |
||
| doCleanupRDD(rdd.id) | ||
| } | ||
|
|
||
| def cleanupShuffle(shuffleDependency: ShuffleDependency[_, _]) { | ||
| doCleanupShuffle(shuffleDependency.shuffleId) | ||
| } | ||
|
|
||
| private def isStopped = stopped | ||
| def cleanupBroadcast[T](broadcast: Broadcast[T]) { | ||
| doCleanupBroadcast(broadcast.id) | ||
| } | ||
| } | ||
|
|
||
| private object ContextCleaner { | ||
| private val REF_QUEUE_POLL_TIMEOUT = 100 | ||
| } | ||
|
|
||
| /** | ||
| * Listener class used for testing when any item has been cleaned by the Cleaner class. | ||
| */ | ||
| private[spark] trait CleanerListener { | ||
| def rddCleaned(rddId: Int) | ||
| def shuffleCleaned(shuffleId: Int) | ||
| def broadcastCleaned(broadcastId: Long) | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -35,7 +35,6 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad | |
| import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} | ||
| import org.apache.mesos.MesosNativeLibrary | ||
|
|
||
| import org.apache.spark.broadcast.Broadcast | ||
| import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} | ||
| import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} | ||
| import org.apache.spark.rdd._ | ||
|
|
@@ -230,6 +229,7 @@ class SparkContext( | |
|
|
||
| private[spark] val cleaner = new ContextCleaner(this) | ||
| cleaner.start() | ||
|
|
||
| postEnvironmentUpdate() | ||
|
|
||
| /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ | ||
|
|
@@ -643,7 +643,11 @@ class SparkContext( | |
| * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. | ||
| * The variable will be sent to each cluster only once. | ||
| */ | ||
| def broadcast[T](value: T): Broadcast[T] = env.broadcastManager.newBroadcast[T](value, isLocal) | ||
| def broadcast[T](value: T) = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It looks like the public return type was dropped, maybe add it back?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. oops |
||
| val bc = env.broadcastManager.newBroadcast[T](value, isLocal) | ||
| cleaner.registerBroadcastForCleanup(bc) | ||
| bc | ||
| } | ||
|
|
||
| /** | ||
| * Add a file to be downloaded with this Spark job on every node. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,9 +18,8 @@ | |
| package org.apache.spark.broadcast | ||
|
|
||
| import java.io.Serializable | ||
| import java.util.concurrent.atomic.AtomicLong | ||
|
|
||
| import org.apache.spark._ | ||
| import org.apache.spark.SparkException | ||
|
|
||
| /** | ||
| * A broadcast variable. Broadcast variables allow the programmer to keep a read-only variable | ||
|
|
@@ -51,49 +50,37 @@ import org.apache.spark._ | |
| * @tparam T Type of the data contained in the broadcast variable. | ||
| */ | ||
| abstract class Broadcast[T](val id: Long) extends Serializable { | ||
| def value: T | ||
|
|
||
| // We cannot have an abstract readObject here due to some weird issues with | ||
| // readObject having to be 'private' in sub-classes. | ||
|
|
||
| override def toString = "Broadcast(" + id + ")" | ||
| } | ||
|
|
||
| private[spark] | ||
| class BroadcastManager(val _isDriver: Boolean, conf: SparkConf, securityManager: SecurityManager) | ||
| extends Logging with Serializable { | ||
|
|
||
| private var initialized = false | ||
| private var broadcastFactory: BroadcastFactory = null | ||
|
|
||
| initialize() | ||
| protected var _isValid: Boolean = true | ||
|
|
||
| // Called by SparkContext or Executor before using Broadcast | ||
| private def initialize() { | ||
| synchronized { | ||
| if (!initialized) { | ||
| val broadcastFactoryClass = conf.get( | ||
| "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") | ||
| /** | ||
| * Whether this Broadcast is actually usable. This should be false once persisted state is | ||
| * removed from the driver. | ||
| */ | ||
| def isValid: Boolean = _isValid | ||
|
|
||
| broadcastFactory = | ||
| Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] | ||
|
|
||
| // Initialize appropriate BroadcastFactory and BroadcastObject | ||
| broadcastFactory.initialize(isDriver, conf, securityManager) | ||
| def value: T | ||
|
|
||
| initialized = true | ||
| } | ||
| /** | ||
| * Remove all persisted state associated with this broadcast on the executors. The next use | ||
| * of this broadcast on the executors will trigger a remote fetch. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This comment uses some terminology that might be opaque to users. What about something like this? |
||
| */ | ||
| def unpersist() | ||
|
|
||
| /** | ||
| * Remove all persisted state associated with this broadcast on both the executors and the | ||
| * driver. Overriding implementations should set isValid to false. | ||
| */ | ||
| private[spark] def destroy() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You could also make this concrete and have a function called This would make it so you don't count on subclasses to deal with the valid bit. |
||
|
|
||
| /** | ||
| * If this broadcast is no longer valid, throw an exception. | ||
| */ | ||
| protected def assertValid() { | ||
| if (!_isValid) { | ||
| throw new SparkException("Attempted to use %s after it has been destroyed!".format(toString)) | ||
| } | ||
| } | ||
|
|
||
| def stop() { | ||
| broadcastFactory.stop() | ||
| } | ||
|
|
||
| private val nextBroadcastId = new AtomicLong(0) | ||
|
|
||
| def newBroadcast[T](value_ : T, isLocal: Boolean) = | ||
| broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) | ||
|
|
||
| def isDriver = _isDriver | ||
| override def toString = "Broadcast(" + id + ")" | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,7 +27,8 @@ import org.apache.spark.SparkConf | |
| * entire Spark job. | ||
| */ | ||
| trait BroadcastFactory { | ||
| def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit | ||
| def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It looks like the only change here was to remove three type signatures. Maybe this was inadvertent from the merge? These should probably all be added back.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Okay. But I am also adding |
||
| def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] | ||
| def stop(): Unit | ||
| def unbroadcast(id: Long, removeFromDriver: Boolean) | ||
| def stop() | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
would be good to set the name of the thread, so that stack dumps are easier to understand.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, I am working on an updated patch based on all the feedback and I have already put that in.