-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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
[SPARK-1103] [WIP] Automatic garbage collection of RDD, shuffle and broadcast data #126
Changes from all 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 |
---|---|---|
@@ -0,0 +1,192 @@ | ||
/* | ||
* 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 | ||
|
||
import java.lang.ref.{ReferenceQueue, WeakReference} | ||
|
||
import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} | ||
|
||
import org.apache.spark.broadcast.Broadcast | ||
import org.apache.spark.rdd.RDD | ||
|
||
/** | ||
* 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 | ||
|
||
/** | ||
* 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 { | ||
|
||
private val referenceBuffer = new ArrayBuffer[CleanupTaskWeakReference] | ||
with SynchronizedBuffer[CleanupTaskWeakReference] | ||
|
||
private val referenceQueue = new ReferenceQueue[AnyRef] | ||
|
||
private val listeners = new ArrayBuffer[CleanerListener] | ||
with SynchronizedBuffer[CleanerListener] | ||
|
||
private val cleaningThread = new Thread() { override def run() { keepCleaning() }} | ||
|
||
/** | ||
* Whether the cleaning thread will block on cleanup tasks. | ||
* This is set to true only for tests. | ||
*/ | ||
private val blockOnCleanupTasks = sc.conf.getBoolean( | ||
"spark.cleaner.referenceTracking.blocking", false) | ||
|
||
@volatile private var stopped = false | ||
|
||
/** Attach a listener object to get information of when objects are cleaned. */ | ||
def attachListener(listener: CleanerListener) { | ||
listeners += listener | ||
} | ||
|
||
/** Start the cleaner. */ | ||
def start() { | ||
cleaningThread.setDaemon(true) | ||
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. 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 commentThe 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. |
||
cleaningThread.setName("Spark Context Cleaner") | ||
cleaningThread.start() | ||
} | ||
|
||
/** Stop the cleaner. */ | ||
def stop() { | ||
stopped = true | ||
} | ||
|
||
/** Register a RDD for cleanup when it is garbage collected. */ | ||
def registerRDDForCleanup(rdd: RDD[_]) { | ||
registerForCleanup(rdd, CleanRDD(rdd.id)) | ||
} | ||
|
||
/** Register a ShuffleDependency for cleanup when it is garbage collected. */ | ||
def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _]) { | ||
registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) | ||
} | ||
|
||
/** Register a Broadcast for cleanup when it is garbage collected. */ | ||
def registerBroadcastForCleanup[T](broadcast: Broadcast[T]) { | ||
registerForCleanup(broadcast, CleanBroadcast(broadcast.id)) | ||
} | ||
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 CleanupTaskWeakReference(task, objectForCleanup, referenceQueue) | ||
} | ||
|
||
/** Keep cleaning RDD, shuffle, and broadcast state. */ | ||
private def keepCleaning() { | ||
while (!stopped) { | ||
try { | ||
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, blocking = blockOnCleanupTasks) | ||
case CleanShuffle(shuffleId) => | ||
doCleanupShuffle(shuffleId, blocking = blockOnCleanupTasks) | ||
case CleanBroadcast(broadcastId) => | ||
doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) | ||
} | ||
} | ||
} catch { | ||
case t: Throwable => logError("Error in cleaning thread", t) | ||
} | ||
} | ||
} | ||
|
||
/** Perform RDD cleanup. */ | ||
def doCleanupRDD(rddId: Int, blocking: Boolean) { | ||
try { | ||
logDebug("Cleaning RDD " + rddId) | ||
sc.unpersistRDD(rddId, blocking) | ||
listeners.foreach(_.rddCleaned(rddId)) | ||
logInfo("Cleaned RDD " + rddId) | ||
} catch { | ||
case t: Throwable => logError("Error cleaning RDD " + rddId, t) | ||
} | ||
} | ||
|
||
/** Perform shuffle cleanup, asynchronously. */ | ||
def doCleanupShuffle(shuffleId: Int, blocking: Boolean) { | ||
try { | ||
logDebug("Cleaning shuffle " + shuffleId) | ||
mapOutputTrackerMaster.unregisterShuffle(shuffleId) | ||
blockManagerMaster.removeShuffle(shuffleId, blocking) | ||
listeners.foreach(_.shuffleCleaned(shuffleId)) | ||
logInfo("Cleaned shuffle " + shuffleId) | ||
} catch { | ||
case t: Throwable => logError("Error cleaning shuffle " + shuffleId, t) | ||
} | ||
} | ||
|
||
/** Perform broadcast cleanup. */ | ||
def doCleanupBroadcast(broadcastId: Long, blocking: Boolean) { | ||
try { | ||
logDebug("Cleaning broadcast " + broadcastId) | ||
broadcastManager.unbroadcast(broadcastId, true, blocking) | ||
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. These methods explicitly blocks until cleanup is completed | ||
// to ensure that more reliable testing. | ||
} | ||
|
||
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) | ||
} |
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.
Why is this a SparkConf instead of just having a utility function called
setBlocking
for tests?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.
That would require a "var", and I generally dont like having a "var" in a class field.