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
Changed ContextCleaner to use ReferenceQueue instead of finalizer
  • Loading branch information
tdas committed Mar 25, 2014
commit f2881fd7d4afaead50632418c4a927ecd09eac65
80 changes: 47 additions & 33 deletions core/src/main/scala/org/apache/spark/ContextCleaner.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand All @@ -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)
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.

Expand All @@ -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. */
Expand All @@ -91,24 +105,23 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
sc.persistentRdds.remove(rddId)
}
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).


/**
* 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)
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)
}
}
} catch {
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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

Expand Down
19 changes: 1 addition & 18 deletions core/src/main/scala/org/apache/spark/Dependency.scala
Original file line number Diff line number Diff line change
Expand Up @@ -53,24 +53,7 @@ class ShuffleDependency[K, V](

val shuffleId: Int = rdd.context.newShuffleId()

override def finalize() {
try {
if (rdd != null) {
rdd.sparkContext.cleaner.scheduleShuffleCleanup(shuffleId)
}
} catch {
case t: Throwable =>
// Paranoia - If logError throws error as well, report to stderr.
try {
logError("Error in finalize", t)
} catch {
case _ : Throwable =>
System.err.println("Error in finalize (and could not write to logError): " + t)
}
} finally {
super.finalize()
}
}
rdd.sparkContext.cleaner.registerShuffleForCleanup(this)
}


Expand Down
18 changes: 1 addition & 17 deletions core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,7 @@ abstract class RDD[T: ClassTag](
}
storageLevel = newLevel
// Register the RDD with the SparkContext
sc.cleaner.registerRDDForCleanup(this)
sc.persistentRdds(id) = this
this
}
Expand Down Expand Up @@ -1102,21 +1103,4 @@ abstract class RDD[T: ClassTag](
def toJavaRDD() : JavaRDD[T] = {
new JavaRDD(this)(elementClassTag)
}

override def finalize() {
try {
sc.cleaner.scheduleRDDCleanup(id)
} catch {
case t: Throwable =>
// Paranoia - If logError throws error as well, report to stderr.
try {
logError("Error in finalize", t)
} catch {
case _ : Throwable =>
System.err.println("Error in finalize (and could not write to logError): " + t)
}
} finally {
super.finalize()
}
}
}
Loading