Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
51 commits
Select commit Hold shift + click to select a range
1e752f1
Added unpersist method to Broadcast.
Feb 5, 2014
80dd977
Fix for Broadcast unpersist patch.
Feb 6, 2014
e427a9e
Added ContextCleaner to automatically clean RDDs and shuffles when th…
tdas Feb 14, 2014
8512612
Changed TimeStampedHashMap to use WrappedJavaHashMap.
tdas Feb 14, 2014
a24fefc
Merge remote-tracking branch 'apache/master' into state-cleanup
tdas Mar 11, 2014
cb0a5a6
Fixed docs and styles.
tdas Mar 11, 2014
ae9da88
Removed unncessary TimeStampedHashMap from DAGScheduler, added try-ca…
tdas Mar 12, 2014
e61daa0
Modifications based on the comments on PR 126.
tdas Mar 13, 2014
a7260d3
Added try-catch in context cleaner and null value cleaning in TimeSta…
tdas Mar 17, 2014
892b952
Removed use of BoundedHashMap, and made BlockManagerSlaveActor cleanu…
tdas Mar 18, 2014
e1fba5f
Style fix
tdas Mar 19, 2014
f2881fd
Changed ContextCleaner to use ReferenceQueue instead of finalizer
tdas Mar 25, 2014
620eca3
Changes based on PR comments.
tdas Mar 25, 2014
a007307
Merge remote-tracking branch 'apache/master' into state-cleanup
tdas Mar 25, 2014
d2f8b97
Removed duplicate unpersistRDD.
tdas Mar 25, 2014
6c9dcf6
Added missing Apache license
tdas Mar 25, 2014
c7ccef1
Merge branch 'bc-unpersist-merge' of github.com:ignatich/incubator-sp…
andrewor14 Mar 26, 2014
ba52e00
Refactor broadcast classes
andrewor14 Mar 26, 2014
d0edef3
Add framework for broadcast cleanup
andrewor14 Mar 26, 2014
544ac86
Clean up broadcast blocks through BlockManager*
andrewor14 Mar 26, 2014
e95479c
Add tests for unpersisting broadcast
andrewor14 Mar 27, 2014
f201a8d
Test broadcast cleanup in ContextCleanerSuite + remove BoundedHashMap
andrewor14 Mar 27, 2014
c92e4d9
Merge github.com:apache/spark into cleanup
andrewor14 Mar 27, 2014
0d17060
Import, comments, and style fixes (minor)
andrewor14 Mar 28, 2014
34f436f
Generalize BroadcastBlockId to remove BroadcastHelperBlockId
andrewor14 Mar 28, 2014
fbfeec8
Add functionality to query executors for their local BlockStatuses
andrewor14 Mar 29, 2014
88904a3
Make TimeStampedWeakValueHashMap a wrapper of TimeStampedHashMap
andrewor14 Mar 29, 2014
e442246
Merge github.com:apache/spark into cleanup
andrewor14 Mar 29, 2014
8557c12
Merge github.com:apache/spark into cleanup
andrewor14 Mar 30, 2014
7edbc98
Merge remote-tracking branch 'apache-github/master' into state-cleanup
tdas Mar 31, 2014
634a097
Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup
andrewor14 Mar 31, 2014
7ed72fb
Fix style test fail + remove verbose test message regarding broadcast
andrewor14 Mar 31, 2014
5016375
Address TD's comments
andrewor14 Apr 1, 2014
f0aabb1
Correct semantics for TimeStampedWeakValueHashMap + add tests
andrewor14 Apr 2, 2014
762a4d8
Merge pull request #1 from andrewor14/cleanup
tdas Apr 2, 2014
a6460d4
Merge github.com:apache/spark into cleanup
andrewor14 Apr 4, 2014
c5b1d98
Address Patrick's comments
andrewor14 Apr 4, 2014
a2cc8bc
Merge remote-tracking branch 'apache/master' into state-cleanup
tdas Apr 4, 2014
ada45f0
Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup
andrewor14 Apr 4, 2014
cd72d19
Make automatic cleanup configurable (not documented)
andrewor14 Apr 4, 2014
b27f8e8
Merge pull request #3 from andrewor14/cleanup
tdas Apr 4, 2014
a430f06
Fixed compilation errors.
tdas Apr 4, 2014
104a89a
Fixed failing BroadcastSuite unit tests by introducing blocking for r…
tdas Apr 4, 2014
6222697
Fixed bug and adding unit test for removeBroadcast in BlockManagerSuite.
tdas Apr 4, 2014
41c9ece
Added more unit tests for BlockManager, DiskBlockManager, and Context…
tdas Apr 7, 2014
2b95b5e
Added more documentation on Broadcast implementations, specially whic…
tdas Apr 7, 2014
4d05314
Scala style fix.
tdas Apr 7, 2014
cff023c
Fixed issues based on Andrew's comments.
tdas Apr 7, 2014
d25a86e
Fixed stupid typo.
tdas Apr 7, 2014
f489fdc
Merge remote-tracking branch 'apache/master' into state-cleanup
tdas Apr 8, 2014
61b8d6e
Fixed issue with Tachyon + new BlockManager methods.
tdas Apr 8, 2014
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Add framework for broadcast cleanup
As of this commit, Spark does not clean up broadcast blocks.
This will be done in the next commit.
  • Loading branch information
andrewor14 committed Mar 26, 2014
commit d0edef3dda333b5bf43a320acd214f276b8a5b3e
134 changes: 86 additions & 48 deletions core/src/main/scala/org/apache/spark/ContextCleaner.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Copy link
Contributor

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.

Copy link
Contributor Author

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.

cleaningThread.setName("ContextCleaner")
Copy link
Contributor

Choose a reason for hiding this comment

The 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()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isn't there a risk that calling interrupt will leave various data-structures in a corrupt/inconsistent state here? From what I can see cleaner.stop() is called before a bunch of other things stop. What if, e.g. the cleaning thread is in the middle of some code that mutates a datastructure in the DAG scheduler or some other data structure that is later stopped... does that ever happen?

Copy link
Contributor

Choose a reason for hiding this comment

The 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 SparkContext and so on.

Copy link
Contributor

Choose a reason for hiding this comment

The 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?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No I'm suggesting that you rely on the existing stopped variable for this and let it cleanly exit the while loop.

}

/**
* 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))
}
Copy link
Contributor

Choose a reason for hiding this comment

The 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)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

From what I can tell here doCleanupRDD is not blocking and the other two are blocking. Is there any ways these could all be non-blocking?

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.

Copy link
Contributor

Choose a reason for hiding this comment

The 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 RemoveShuffle docs that it is nonblocking (i.e. it just creates requests to remove the shuffle on the executors).

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)
}
}
Expand All @@ -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 {
Expand All @@ -150,10 +151,47 @@ 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

private[spark] def cleanupRDD(rdd: RDD[_]) {
doCleanupRDD(rdd.id)
}

private[spark] def cleanupShuffle(shuffleDependency: ShuffleDependency[_, _]) {
doCleanupShuffle(shuffleDependency.shuffleId)
}

private def isStopped = stopped
private[spark] 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)
}
6 changes: 5 additions & 1 deletion core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -642,7 +642,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) = env.broadcastManager.newBroadcast[T](value, isLocal)
def broadcast[T](value: T) = {
Copy link
Contributor

Choose a reason for hiding this comment

The 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?

Copy link
Contributor

Choose a reason for hiding this comment

The 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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,12 @@ import java.io.Serializable
abstract class Broadcast[T](val id: Long) extends Serializable {
def value: T

/**
* Remove all persisted state associated with this broadcast.
* @param removeFromDriver Whether to remove state from the driver.
*/
def unpersist(removeFromDriver: Boolean)

// We cannot have an abstract readObject here due to some weird issues with
// readObject having to be 'private' in sub-classes.

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,5 +29,6 @@ import org.apache.spark.SparkConf
trait BroadcastFactory {
def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit
def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T]
def unbroadcast(id: Long, removeFromDriver: Boolean)
def stop(): Unit
}
Original file line number Diff line number Diff line change
Expand Up @@ -60,4 +60,8 @@ private[spark] class BroadcastManager(
broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement())
}

def unbroadcast(id: Long, removeFromDriver: Boolean) {
broadcastFactory.unbroadcast(id, removeFromDriver)
}

}
Loading