-
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 1 commit
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
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,10 +17,11 @@ | |
|
|
||
| package org.apache.spark | ||
|
|
||
| import java.lang.ref.{ReferenceQueue, WeakReference} | ||
|
|
||
| import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} | ||
|
|
||
| import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} | ||
| import org.apache.spark.storage.StorageLevel | ||
| 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 { | ||
|
|
@@ -34,20 +35,27 @@ private[spark] trait CleanerListener { | |
| private[spark] class ContextCleaner(sc: SparkContext) extends Logging { | ||
|
|
||
| /** Classes to represent cleaning tasks */ | ||
| private sealed trait CleaningTask | ||
| private case class CleanRDD(rddId: Int) extends CleaningTask | ||
| private case class CleanShuffle(shuffleId: Int) extends CleaningTask | ||
| 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 queue = new LinkedBlockingQueue[CleaningTask] | ||
| private val referenceBuffer = new ArrayBuffer[WeakReferenceWithCleanupTask] | ||
| with SynchronizedBuffer[WeakReferenceWithCleanupTask] | ||
| private val referenceQueue = new ReferenceQueue[AnyRef] | ||
|
|
||
| protected val listeners = new ArrayBuffer[CleanerListener] | ||
| 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) | ||
|
|
||
| /** Start the cleaner */ | ||
| def start() { | ||
| cleaningThread.setDaemon(true) | ||
|
|
@@ -62,21 +70,27 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { | |
| } | ||
|
|
||
| /** | ||
| * Schedule cleanup of RDD data. Do not perform any time or resource intensive | ||
| * computation in this function as this is called from a finalize() function. | ||
| * Register a RDD for cleanup when it is garbage collected. | ||
| */ | ||
| def scheduleRDDCleanup(rddId: Int) { | ||
| enqueue(CleanRDD(rddId)) | ||
| logDebug("Enqueued RDD " + rddId + " for cleaning up") | ||
| def registerRDDForCleanup(rdd: RDD[_]) { | ||
| registerForCleanup(rdd, CleanRDD(rdd.id)) | ||
| } | ||
|
|
||
| /** | ||
| * Schedule cleanup of shuffle data. Do not perform any time or resource intensive | ||
| * computation in this function as this is called from a finalize() function. | ||
| * Register a shuffle dependency for cleanup when it is garbage collected. | ||
| */ | ||
| def scheduleShuffleCleanup(shuffleId: Int) { | ||
| enqueue(CleanShuffle(shuffleId)) | ||
| logDebug("Enqueued shuffle " + shuffleId + " for cleaning up") | ||
| 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. */ | ||
|
|
@@ -91,24 +105,23 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { | |
| sc.persistentRdds.remove(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. 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). |
||
|
|
||
| /** | ||
| * Enqueue a cleaning task. Do not perform any time or resource intensive | ||
| * computation in this function as this is called from a finalize() function. | ||
| */ | ||
| private def enqueue(task: CleaningTask) { | ||
| queue.put(task) | ||
| /** Register an object for cleanup. */ | ||
| private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask) { | ||
| referenceBuffer += new WeakReferenceWithCleanupTask(objectForCleanup, task) | ||
| } | ||
|
|
||
| /** Keep cleaning RDDs and shuffle data */ | ||
| private def keepCleaning() { | ||
| while (!isStopped) { | ||
| try { | ||
| val taskOpt = Option(queue.poll(100, TimeUnit.MILLISECONDS)) | ||
| taskOpt.foreach { task => | ||
| logDebug("Got cleaning task " + taskOpt.get) | ||
| val reference = Option(referenceQueue.remove(REF_QUEUE_POLL_TIMEOUT)) | ||
| .map(_.asInstanceOf[WeakReferenceWithCleanupTask]) | ||
| reference.map(_.task).foreach { task => | ||
| logDebug("Got cleaning task " + task) | ||
| referenceBuffer -= reference.get | ||
| task match { | ||
| case CleanRDD(rddId) => doCleanRDD(rddId) | ||
| case CleanShuffle(shuffleId) => doCleanShuffle(shuffleId) | ||
| 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) | ||
| } | ||
| } | ||
| } catch { | ||
|
|
@@ -119,8 +132,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { | |
| } | ||
| } | ||
|
|
||
| /** Perform RDD cleaning */ | ||
| private def doCleanRDD(rddId: Int) { | ||
| /** Perform RDD cleanup. */ | ||
| private def doCleanupRDD(rddId: Int) { | ||
| try { | ||
| logDebug("Cleaning RDD " + rddId) | ||
| unpersistRDD(rddId, false) | ||
|
|
@@ -131,8 +144,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { | |
| } | ||
| } | ||
|
|
||
| /** Perform shuffle cleaning */ | ||
| private def doCleanShuffle(shuffleId: Int) { | ||
| /** Perform shuffle cleanup. */ | ||
| private def doCleanupShuffle(shuffleId: Int) { | ||
| try { | ||
| logDebug("Cleaning shuffle " + shuffleId) | ||
| mapOutputTrackerMaster.unregisterShuffle(shuffleId) | ||
|
|
@@ -144,7 +157,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { | |
| } | ||
| } | ||
|
|
||
| private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] | ||
| private def mapOutputTrackerMaster = | ||
| sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] | ||
|
|
||
| private def blockManagerMaster = sc.env.blockManager.master | ||
|
|
||
|
|
||
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.