@@ -21,105 +21,106 @@ import java.lang.ref.{ReferenceQueue, WeakReference}
21
21
22
22
import scala .collection .mutable .{ArrayBuffer , SynchronizedBuffer }
23
23
24
+ import org .apache .spark .broadcast .Broadcast
24
25
import org .apache .spark .rdd .RDD
25
26
26
- /** Listener class used for testing when any item has been cleaned by the Cleaner class */
27
- private [spark] trait CleanerListener {
28
- def rddCleaned (rddId : Int )
29
- def shuffleCleaned (shuffleId : Int )
30
- }
27
+ /**
28
+ * Classes that represent cleaning tasks.
29
+ */
30
+ private sealed trait CleanupTask
31
+ private case class CleanRDD (rddId : Int ) extends CleanupTask
32
+ private case class CleanShuffle (shuffleId : Int ) extends CleanupTask
33
+ private case class CleanBroadcast (broadcastId : Long ) extends CleanupTask
31
34
32
35
/**
33
- * Cleans RDDs and shuffle data.
36
+ * A WeakReference associated with a CleanupTask.
37
+ *
38
+ * When the referent object becomes only weakly reachable, the corresponding
39
+ * CleanupTaskWeakReference is automatically added to the given reference queue.
40
+ */
41
+ private class CleanupTaskWeakReference (
42
+ val task : CleanupTask ,
43
+ referent : AnyRef ,
44
+ referenceQueue : ReferenceQueue [AnyRef ])
45
+ extends WeakReference (referent, referenceQueue)
46
+
47
+ /**
48
+ * An asynchronous cleaner for RDD, shuffle, and broadcast state.
49
+ *
50
+ * This maintains a weak reference for each RDD, ShuffleDependency, and Broadcast of interest,
51
+ * to be processed when the associated object goes out of scope of the application. Actual
52
+ * cleanup is performed in a separate daemon thread.
34
53
*/
35
54
private [spark] class ContextCleaner (sc : SparkContext ) extends Logging {
36
55
37
- /** Classes to represent cleaning tasks */
38
- private sealed trait CleanupTask
39
- private case class CleanRDD (rddId : Int ) extends CleanupTask
40
- private case class CleanShuffle (shuffleId : Int ) extends CleanupTask
41
- // TODO: add CleanBroadcast
56
+ private val referenceBuffer = new ArrayBuffer [CleanupTaskWeakReference ]
57
+ with SynchronizedBuffer [CleanupTaskWeakReference ]
42
58
43
- private val referenceBuffer = new ArrayBuffer [WeakReferenceWithCleanupTask ]
44
- with SynchronizedBuffer [WeakReferenceWithCleanupTask ]
45
59
private val referenceQueue = new ReferenceQueue [AnyRef ]
46
60
47
61
private val listeners = new ArrayBuffer [CleanerListener ]
48
62
with SynchronizedBuffer [CleanerListener ]
49
63
50
64
private val cleaningThread = new Thread () { override def run () { keepCleaning() }}
51
65
52
- private val REF_QUEUE_POLL_TIMEOUT = 100
53
-
54
66
@ volatile private var stopped = false
55
67
56
- private class WeakReferenceWithCleanupTask (referent : AnyRef , val task : CleanupTask )
57
- extends WeakReference (referent, referenceQueue)
68
+ /** Attach a listener object to get information of when objects are cleaned. */
69
+ def attachListener (listener : CleanerListener ) {
70
+ listeners += listener
71
+ }
58
72
59
- /** Start the cleaner */
73
+ /** Start the cleaner. */
60
74
def start () {
61
75
cleaningThread.setDaemon(true )
62
76
cleaningThread.setName(" ContextCleaner" )
63
77
cleaningThread.start()
64
78
}
65
79
66
- /** Stop the cleaner */
80
+ /** Stop the cleaner. */
67
81
def stop () {
68
82
stopped = true
69
83
cleaningThread.interrupt()
70
84
}
71
85
72
- /**
73
- * Register a RDD for cleanup when it is garbage collected.
74
- */
86
+ /** Register a RDD for cleanup when it is garbage collected. */
75
87
def registerRDDForCleanup (rdd : RDD [_]) {
76
88
registerForCleanup(rdd, CleanRDD (rdd.id))
77
89
}
78
90
79
- /**
80
- * Register a shuffle dependency for cleanup when it is garbage collected.
81
- */
91
+ /** Register a ShuffleDependency for cleanup when it is garbage collected. */
82
92
def registerShuffleForCleanup (shuffleDependency : ShuffleDependency [_, _]) {
83
93
registerForCleanup(shuffleDependency, CleanShuffle (shuffleDependency.shuffleId))
84
94
}
85
95
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)
94
- }
95
-
96
- /** Attach a listener object to get information of when objects are cleaned. */
97
- def attachListener (listener : CleanerListener ) {
98
- listeners += listener
96
+ /** Register a Broadcast for cleanup when it is garbage collected. */
97
+ def registerBroadcastForCleanup [T ](broadcast : Broadcast [T ]) {
98
+ registerForCleanup(broadcast, CleanBroadcast (broadcast.id))
99
99
}
100
100
101
101
/** Register an object for cleanup. */
102
102
private def registerForCleanup (objectForCleanup : AnyRef , task : CleanupTask ) {
103
- referenceBuffer += new WeakReferenceWithCleanupTask ( objectForCleanup, task )
103
+ referenceBuffer += new CleanupTaskWeakReference (task, objectForCleanup, referenceQueue )
104
104
}
105
105
106
- /** Keep cleaning RDDs and shuffle data */
106
+ /** Keep cleaning RDD, shuffle, and broadcast state. */
107
107
private def keepCleaning () {
108
- while (! isStopped ) {
108
+ while (! stopped ) {
109
109
try {
110
- val reference = Option (referenceQueue.remove(REF_QUEUE_POLL_TIMEOUT ))
111
- .map(_.asInstanceOf [WeakReferenceWithCleanupTask ])
110
+ val reference = Option (referenceQueue.remove(ContextCleaner . REF_QUEUE_POLL_TIMEOUT ))
111
+ .map(_.asInstanceOf [CleanupTaskWeakReference ])
112
112
reference.map(_.task).foreach { task =>
113
113
logDebug(" Got cleaning task " + task)
114
114
referenceBuffer -= reference.get
115
115
task match {
116
116
case CleanRDD (rddId) => doCleanupRDD(rddId)
117
117
case CleanShuffle (shuffleId) => doCleanupShuffle(shuffleId)
118
+ case CleanBroadcast (broadcastId) => doCleanupBroadcast(broadcastId)
118
119
}
119
120
}
120
121
} catch {
121
122
case ie : InterruptedException =>
122
- if (! isStopped ) logWarning(" Cleaning thread interrupted" )
123
+ if (! stopped ) logWarning(" Cleaning thread interrupted" )
123
124
case t : Throwable => logError(" Error in cleaning thread" , t)
124
125
}
125
126
}
@@ -129,7 +130,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
129
130
private def doCleanupRDD (rddId : Int ) {
130
131
try {
131
132
logDebug(" Cleaning RDD " + rddId)
132
- sc.unpersistRDD(rddId, false )
133
+ sc.unpersistRDD(rddId, blocking = false )
133
134
listeners.foreach(_.rddCleaned(rddId))
134
135
logInfo(" Cleaned RDD " + rddId)
135
136
} catch {
@@ -150,10 +151,47 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
150
151
}
151
152
}
152
153
153
- private def mapOutputTrackerMaster =
154
- sc.env.mapOutputTracker.asInstanceOf [MapOutputTrackerMaster ]
154
+ /** Perform broadcast cleanup. */
155
+ private def doCleanupBroadcast (broadcastId : Long ) {
156
+ try {
157
+ logDebug(" Cleaning broadcast " + broadcastId)
158
+ broadcastManager.unbroadcast(broadcastId, removeFromDriver = true )
159
+ listeners.foreach(_.broadcastCleaned(broadcastId))
160
+ logInfo(" Cleaned broadcast " + broadcastId)
161
+ } catch {
162
+ case t : Throwable => logError(" Error cleaning broadcast " + broadcastId, t)
163
+ }
164
+ }
155
165
156
166
private def blockManagerMaster = sc.env.blockManager.master
167
+ private def broadcastManager = sc.env.broadcastManager
168
+ private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf [MapOutputTrackerMaster ]
169
+
170
+ // Used for testing
171
+
172
+ private [spark] def cleanupRDD (rdd : RDD [_]) {
173
+ doCleanupRDD(rdd.id)
174
+ }
175
+
176
+ private [spark] def cleanupShuffle (shuffleDependency : ShuffleDependency [_, _]) {
177
+ doCleanupShuffle(shuffleDependency.shuffleId)
178
+ }
157
179
158
- private def isStopped = stopped
180
+ private [spark] def cleanupBroadcast [T ](broadcast : Broadcast [T ]) {
181
+ doCleanupBroadcast(broadcast.id)
182
+ }
183
+
184
+ }
185
+
186
+ private object ContextCleaner {
187
+ private val REF_QUEUE_POLL_TIMEOUT = 100
188
+ }
189
+
190
+ /**
191
+ * Listener class used for testing when any item has been cleaned by the Cleaner class.
192
+ */
193
+ private [spark] trait CleanerListener {
194
+ def rddCleaned (rddId : Int )
195
+ def shuffleCleaned (shuffleId : Int )
196
+ def broadcastCleaned (broadcastId : Long )
159
197
}
0 commit comments