Skip to content

Commit f2881fd

Browse files
committed
Changed ContextCleaner to use ReferenceQueue instead of finalizer
1 parent e1fba5f commit f2881fd

File tree

4 files changed

+123
-94
lines changed

4 files changed

+123
-94
lines changed

core/src/main/scala/org/apache/spark/ContextCleaner.scala

Lines changed: 47 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -17,10 +17,11 @@
1717

1818
package org.apache.spark
1919

20+
import java.lang.ref.{ReferenceQueue, WeakReference}
21+
2022
import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
2123

22-
import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
23-
import org.apache.spark.storage.StorageLevel
24+
import org.apache.spark.rdd.RDD
2425

2526
/** Listener class used for testing when any item has been cleaned by the Cleaner class */
2627
private[spark] trait CleanerListener {
@@ -34,20 +35,27 @@ private[spark] trait CleanerListener {
3435
private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
3536

3637
/** Classes to represent cleaning tasks */
37-
private sealed trait CleaningTask
38-
private case class CleanRDD(rddId: Int) extends CleaningTask
39-
private case class CleanShuffle(shuffleId: Int) extends CleaningTask
38+
private sealed trait CleanupTask
39+
private case class CleanRDD(rddId: Int) extends CleanupTask
40+
private case class CleanShuffle(shuffleId: Int) extends CleanupTask
4041
// TODO: add CleanBroadcast
4142

42-
private val queue = new LinkedBlockingQueue[CleaningTask]
43+
private val referenceBuffer = new ArrayBuffer[WeakReferenceWithCleanupTask]
44+
with SynchronizedBuffer[WeakReferenceWithCleanupTask]
45+
private val referenceQueue = new ReferenceQueue[AnyRef]
4346

44-
protected val listeners = new ArrayBuffer[CleanerListener]
47+
private val listeners = new ArrayBuffer[CleanerListener]
4548
with SynchronizedBuffer[CleanerListener]
4649

4750
private val cleaningThread = new Thread() { override def run() { keepCleaning() }}
4851

52+
private val REF_QUEUE_POLL_TIMEOUT = 100
53+
4954
@volatile private var stopped = false
5055

56+
private class WeakReferenceWithCleanupTask(referent: AnyRef, val task: CleanupTask)
57+
extends WeakReference(referent, referenceQueue)
58+
5159
/** Start the cleaner */
5260
def start() {
5361
cleaningThread.setDaemon(true)
@@ -62,21 +70,27 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
6270
}
6371

6472
/**
65-
* Schedule cleanup of RDD data. Do not perform any time or resource intensive
66-
* computation in this function as this is called from a finalize() function.
73+
* Register a RDD for cleanup when it is garbage collected.
6774
*/
68-
def scheduleRDDCleanup(rddId: Int) {
69-
enqueue(CleanRDD(rddId))
70-
logDebug("Enqueued RDD " + rddId + " for cleaning up")
75+
def registerRDDForCleanup(rdd: RDD[_]) {
76+
registerForCleanup(rdd, CleanRDD(rdd.id))
7177
}
7278

7379
/**
74-
* Schedule cleanup of shuffle data. Do not perform any time or resource intensive
75-
* computation in this function as this is called from a finalize() function.
80+
* Register a shuffle dependency for cleanup when it is garbage collected.
7681
*/
77-
def scheduleShuffleCleanup(shuffleId: Int) {
78-
enqueue(CleanShuffle(shuffleId))
79-
logDebug("Enqueued shuffle " + shuffleId + " for cleaning up")
82+
def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _]) {
83+
registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId))
84+
}
85+
86+
/** Cleanup RDD. */
87+
def cleanupRDD(rdd: RDD[_]) {
88+
doCleanupRDD(rdd.id)
89+
}
90+
91+
/** Cleanup shuffle. */
92+
def cleanupShuffle(shuffleDependency: ShuffleDependency[_, _]) {
93+
doCleanupShuffle(shuffleDependency.shuffleId)
8094
}
8195

8296
/** Attach a listener object to get information of when objects are cleaned. */
@@ -91,24 +105,23 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
91105
sc.persistentRdds.remove(rddId)
92106
}
93107

94-
/**
95-
* Enqueue a cleaning task. Do not perform any time or resource intensive
96-
* computation in this function as this is called from a finalize() function.
97-
*/
98-
private def enqueue(task: CleaningTask) {
99-
queue.put(task)
108+
/** Register an object for cleanup. */
109+
private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask) {
110+
referenceBuffer += new WeakReferenceWithCleanupTask(objectForCleanup, task)
100111
}
101112

102113
/** Keep cleaning RDDs and shuffle data */
103114
private def keepCleaning() {
104115
while (!isStopped) {
105116
try {
106-
val taskOpt = Option(queue.poll(100, TimeUnit.MILLISECONDS))
107-
taskOpt.foreach { task =>
108-
logDebug("Got cleaning task " + taskOpt.get)
117+
val reference = Option(referenceQueue.remove(REF_QUEUE_POLL_TIMEOUT))
118+
.map(_.asInstanceOf[WeakReferenceWithCleanupTask])
119+
reference.map(_.task).foreach { task =>
120+
logDebug("Got cleaning task " + task)
121+
referenceBuffer -= reference.get
109122
task match {
110-
case CleanRDD(rddId) => doCleanRDD(rddId)
111-
case CleanShuffle(shuffleId) => doCleanShuffle(shuffleId)
123+
case CleanRDD(rddId) => doCleanupRDD(rddId)
124+
case CleanShuffle(shuffleId) => doCleanupShuffle(shuffleId)
112125
}
113126
}
114127
} catch {
@@ -119,8 +132,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
119132
}
120133
}
121134

122-
/** Perform RDD cleaning */
123-
private def doCleanRDD(rddId: Int) {
135+
/** Perform RDD cleanup. */
136+
private def doCleanupRDD(rddId: Int) {
124137
try {
125138
logDebug("Cleaning RDD " + rddId)
126139
unpersistRDD(rddId, false)
@@ -131,8 +144,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
131144
}
132145
}
133146

134-
/** Perform shuffle cleaning */
135-
private def doCleanShuffle(shuffleId: Int) {
147+
/** Perform shuffle cleanup. */
148+
private def doCleanupShuffle(shuffleId: Int) {
136149
try {
137150
logDebug("Cleaning shuffle " + shuffleId)
138151
mapOutputTrackerMaster.unregisterShuffle(shuffleId)
@@ -144,7 +157,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
144157
}
145158
}
146159

147-
private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]
160+
private def mapOutputTrackerMaster =
161+
sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]
148162

149163
private def blockManagerMaster = sc.env.blockManager.master
150164

core/src/main/scala/org/apache/spark/Dependency.scala

Lines changed: 1 addition & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -53,24 +53,7 @@ class ShuffleDependency[K, V](
5353

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

56-
override def finalize() {
57-
try {
58-
if (rdd != null) {
59-
rdd.sparkContext.cleaner.scheduleShuffleCleanup(shuffleId)
60-
}
61-
} catch {
62-
case t: Throwable =>
63-
// Paranoia - If logError throws error as well, report to stderr.
64-
try {
65-
logError("Error in finalize", t)
66-
} catch {
67-
case _ : Throwable =>
68-
System.err.println("Error in finalize (and could not write to logError): " + t)
69-
}
70-
} finally {
71-
super.finalize()
72-
}
73-
}
56+
rdd.sparkContext.cleaner.registerShuffleForCleanup(this)
7457
}
7558

7659

core/src/main/scala/org/apache/spark/rdd/RDD.scala

Lines changed: 1 addition & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -147,6 +147,7 @@ abstract class RDD[T: ClassTag](
147147
}
148148
storageLevel = newLevel
149149
// Register the RDD with the SparkContext
150+
sc.cleaner.registerRDDForCleanup(this)
150151
sc.persistentRdds(id) = this
151152
this
152153
}
@@ -1102,21 +1103,4 @@ abstract class RDD[T: ClassTag](
11021103
def toJavaRDD() : JavaRDD[T] = {
11031104
new JavaRDD(this)(elementClassTag)
11041105
}
1105-
1106-
override def finalize() {
1107-
try {
1108-
sc.cleaner.scheduleRDDCleanup(id)
1109-
} catch {
1110-
case t: Throwable =>
1111-
// Paranoia - If logError throws error as well, report to stderr.
1112-
try {
1113-
logError("Error in finalize", t)
1114-
} catch {
1115-
case _ : Throwable =>
1116-
System.err.println("Error in finalize (and could not write to logError): " + t)
1117-
}
1118-
} finally {
1119-
super.finalize()
1120-
}
1121-
}
11221106
}

0 commit comments

Comments
 (0)