-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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 5 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,126 @@ | ||
/* | ||
* 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 scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} | ||
|
||
import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} | ||
|
||
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) | ||
} | ||
|
||
/** | ||
* Cleans RDDs and shuffle data. | ||
*/ | ||
private[spark] class ContextCleaner(env: SparkEnv) extends Logging { | ||
|
||
/** Classes to represent cleaning tasks */ | ||
private sealed trait CleaningTask | ||
private case class CleanRDD(sc: SparkContext, id: Int) extends CleaningTask | ||
private case class CleanShuffle(id: Int) extends CleaningTask | ||
// TODO: add CleanBroadcast | ||
|
||
private val queue = new LinkedBlockingQueue[CleaningTask] | ||
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. Should this be synchronized? 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. BlockingQueue implementations are thread-safe. :) 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. What's the rationale to do this as an explicit BlockingQueue and polling Thread instead of as an actor? 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. The rationale behind this was to have a tight control over what code gets invoked from a finalize function. Using actor would also work, but given the higher risk code path of being called from the finalize function (RDD.finalize --> RDD.cleanup --> ContextCleaner.cleadRDD --> enqueue RDD ID in the queue), I think its better to use more easily understandable components. |
||
|
||
protected val listeners = new ArrayBuffer[CleanerListener] | ||
with SynchronizedBuffer[CleanerListener] | ||
|
||
private val cleaningThread = new Thread() { override def run() { keepCleaning() }} | ||
|
||
private var stopped = false | ||
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. Any particular reason not to use volatile instead of explicit synchronization here? It's not a big deal either way, the performance difference should be negligible here... 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. No particular reason really. Maybe I am wrong, but I dont think volatile is used much in the non-test code base, so I didnt use it. I will change it volatile, probably slightly cleaner code. |
||
|
||
/** 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.start() | ||
} | ||
|
||
/** Stop the cleaner */ | ||
def stop() { | ||
synchronized { stopped = true } | ||
cleaningThread.interrupt() | ||
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. Isn't there a risk that calling 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. Ya it looks like a lot of the clean-up implementations interact with shared state like the 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. Are you suggesting that we just let the thread terminate automatically with the JVM? 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. No I'm suggesting that you rely on the existing |
||
} | ||
|
||
/** Clean (unpersist) RDD data. */ | ||
def cleanRDD(rdd: RDD[_]) { | ||
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. This may be paranoia speaking, but it would seem cleaner to me to pass in only the fields of RDD that you need here, because calling cleanRDD(this) in a finalizer seems sketchy. It's clearly not a problem right now, but just for future modifiers... 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. I did this because I need both the RDD ID and the associated SparkContext for cleanup and its seems too specific to pass them separately. What if in the future we need some other field of the RDD? 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. I'm only concerned with someone accidentally persisting a reference to the RDD somewhere, which would either halt finalization or destroy the universe. |
||
enqueue(CleanRDD(rdd.sparkContext, rdd.id)) | ||
logDebug("Enqueued RDD " + rdd + " for cleaning up") | ||
} | ||
|
||
/** Clean shuffle data. */ | ||
def cleanShuffle(shuffleId: Int) { | ||
enqueue(CleanShuffle(shuffleId)) | ||
logDebug("Enqueued shuffle " + shuffleId + " for cleaning up") | ||
} | ||
|
||
def attachListener(listener: CleanerListener) { | ||
listeners += listener | ||
} | ||
/** Enqueue a cleaning task */ | ||
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. nit: newline before this line |
||
private def enqueue(task: CleaningTask) { | ||
queue.put(task) | ||
} | ||
|
||
/** Keep cleaning RDDs and shuffle data */ | ||
private def keepCleaning() { | ||
try { | ||
while (!isStopped) { | ||
val taskOpt = Option(queue.poll(100, TimeUnit.MILLISECONDS)) | ||
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. We should make this configurable 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. I don't think this necessarily needs to be configurable, simply because the "when" of a finalizer is always unclear, so changing the period of checking won't really give you any new guarantees. 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. Yeah, @aarondav answered it for me. 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. By configurable I don't mean exposing it to the user. It just seems to be that 100ms is completely arbitrary and it would be good to at least have a mechanism to change it. Though either way this is not a big deal. |
||
if (taskOpt.isDefined) { | ||
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. What do you think about this?
could also include a few monads and higher order functions while we're at it, though 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. Or |
||
logDebug("Got cleaning task " + taskOpt.get) | ||
taskOpt.get match { | ||
case CleanRDD(sc, rddId) => doCleanRDD(sc, rddId) | ||
case CleanShuffle(shuffleId) => doCleanShuffle(shuffleId) | ||
} | ||
} | ||
} | ||
} catch { | ||
case ie: java.lang.InterruptedException => | ||
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. Why the fully qualified path? It doesn't even need an import! |
||
if (!isStopped) logWarning("Cleaning thread interrupted") | ||
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. wrap the body in curly braces 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. Which body? The whole "if (!stopped) ... " or the part after the if clause? 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. latter |
||
} | ||
} | ||
|
||
/** Perform RDD cleaning */ | ||
private def doCleanRDD(sc: SparkContext, rddId: Int) { | ||
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 it make more sense to just store the SparkContext passed to the constructor of
Otherwise it seems a little odd here to pass a SparkContext again. 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. Already fixed that. Left over code. Dont need |
||
logDebug("Cleaning rdd " + rddId) | ||
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. Why doesn't this just call 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. Because the RDD object isnt available here (already garbage collected). Yes, I am aware that this is a duplicate code path. Can you think of a better place (that is, not in RDD class) where we can move this functionality? 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. what about if the RDD unpersist function called 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. That can done. The only thing is that cleaning up RDD may in future involve more than just unpersisting. |
||
sc.env.blockManager.master.removeRdd(rddId, false) | ||
sc.persistentRdds.remove(rddId) | ||
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. It seems that if 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. Minor: Also, you already defined a shorthand to the block manager below, so you might as well use it. |
||
listeners.foreach(_.rddCleaned(rddId)) | ||
logInfo("Cleaned rdd " + rddId) | ||
} | ||
|
||
/** Perform shuffle cleaning */ | ||
private def doCleanShuffle(shuffleId: Int) { | ||
logDebug("Cleaning shuffle " + shuffleId) | ||
mapOutputTrackerMaster.unregisterShuffle(shuffleId) | ||
blockManager.master.removeShuffle(shuffleId) | ||
listeners.foreach(_.shuffleCleaned(shuffleId)) | ||
logInfo("Cleaned shuffle " + shuffleId) | ||
} | ||
|
||
private def mapOutputTrackerMaster = env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] | ||
|
||
private def blockManager = env.blockManager | ||
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. Minor: looks like you only ever use this with |
||
|
||
private def isStopped = synchronized { stopped } | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -49,9 +49,28 @@ class ShuffleDependency[K, V]( | |
@transient rdd: RDD[_ <: Product2[K, V]], | ||
val partitioner: Partitioner, | ||
val serializerClass: String = null) | ||
extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { | ||
extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) with Logging { | ||
|
||
val shuffleId: Int = rdd.context.newShuffleId() | ||
|
||
override def finalize() { | ||
try { | ||
if (rdd != null) { | ||
rdd.sparkContext.cleaner.cleanShuffle(shuffleId) | ||
} | ||
} catch { | ||
case t: Throwable => | ||
// Paranoia - If logError throws error as well, report to stderr. | ||
try { | ||
logError("Error in finalize", t) | ||
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. @tdas Hey TD, A try catch on logging ? 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. Maybe it is a overkill. I am just trying to make sure that no exception gets propagated through the finalizing thread (i.e., the Java system thread that calls finalize on objects) and its probably best to not allow exceptions being thrown on that thread. 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. catching Exception would be sufficient even for paranoia :-) |
||
} catch { | ||
case _ => | ||
System.err.println("Error in finalize (and could not write to logError): " + t) | ||
} | ||
} finally { | ||
super.finalize() | ||
} | ||
} | ||
} | ||
|
||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,10 @@ | |
|
||
package org.apache.spark | ||
|
||
import scala.Some | ||
import scala.collection.mutable.{HashSet, Map} | ||
import scala.concurrent.Await | ||
|
||
import java.io._ | ||
import java.util.zip.{GZIPInputStream, GZIPOutputStream} | ||
|
||
|
@@ -28,7 +32,7 @@ import akka.pattern.ask | |
|
||
import org.apache.spark.scheduler.MapStatus | ||
import org.apache.spark.storage.BlockManagerId | ||
import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} | ||
import org.apache.spark.util.{AkkaUtils, TimeStampedHashMap, BoundedHashMap} | ||
|
||
private[spark] sealed trait MapOutputTrackerMessage | ||
private[spark] case class GetMapOutputStatuses(shuffleId: Int) | ||
|
@@ -50,23 +54,26 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster | |
} | ||
} | ||
|
||
private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { | ||
/** | ||
* Class that keeps track of the location of the location of the mapt output of | ||
* a stage. This is abstract because different versions of MapOutputTracker | ||
* (driver and worker) use different HashMap to store its metadata. | ||
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. "... keeps track of the location of the location of the mapt output..." 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. Fixed. |
||
*/ | ||
private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { | ||
|
||
private val timeout = AkkaUtils.askTimeout(conf) | ||
|
||
// Set to the MapOutputTrackerActor living on the driver | ||
var trackerActor: ActorRef = _ | ||
|
||
protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] | ||
/** This HashMap needs to have different storage behavior for driver and worker */ | ||
protected val mapStatuses: Map[Int, Array[MapStatus]] | ||
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. This comment is a bit vague - could you elaborate on what is different? |
||
|
||
// Incremented every time a fetch fails so that client nodes know to clear | ||
// their cache of map output locations if this happens. | ||
protected var epoch: Long = 0 | ||
protected val epochLock = new java.lang.Object | ||
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. nit: not part of your patch, but can this just be AnyRef? 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. Yeah, does the exact same thing. Changed. |
||
|
||
private val metadataCleaner = | ||
new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) | ||
|
||
// Send a message to the trackerActor and get its result within a default timeout, or | ||
// throw a SparkException if this fails. | ||
private def askTracker(message: Any): Any = { | ||
|
@@ -137,8 +144,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { | |
fetchedStatuses.synchronized { | ||
return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses) | ||
} | ||
} | ||
else { | ||
} else { | ||
throw new FetchFailedException(null, shuffleId, -1, reduceId, | ||
new Exception("Missing all output locations for shuffle " + shuffleId)) | ||
} | ||
|
@@ -150,13 +156,12 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { | |
} | ||
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. Not exactly related to your patch, but does the MOTMaster ever call 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. Makes sense. Will do. 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. I played around with this and actually realized that the these are used in the local mode. So they kind of have to be in the MapOutputTracker as both MapOutputTrackerWorker and MapOutputTrackerMaster (in local mode), needs them. |
||
|
||
protected def cleanup(cleanupTime: Long) { | ||
mapStatuses.clearOldValues(cleanupTime) | ||
mapStatuses.asInstanceOf[TimeStampedHashMap[_, _]].clearOldValues(cleanupTime) | ||
} | ||
|
||
def stop() { | ||
communicate(StopMapOutputTracker) | ||
mapStatuses.clear() | ||
metadataCleaner.cancel() | ||
trackerActor = null | ||
} | ||
|
||
|
@@ -181,15 +186,49 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { | |
} | ||
} | ||
|
||
/** | ||
* MapOutputTracker for the workers. This uses BoundedHashMap to keep track of | ||
* a limited number of most recently used map output information. | ||
*/ | ||
private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { | ||
|
||
/** | ||
* Bounded HashMap for storing serialized statuses in the worker. This allows | ||
* the HashMap stay bounded in memory-usage. Things dropped from this HashMap will be | ||
* automatically repopulated by fetching them again from the driver. | ||
*/ | ||
protected val MAX_MAP_STATUSES = 100 | ||
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. Is this arbitrary? These bounds appear in other places too. Maybe we should make them configurable too. |
||
protected val mapStatuses = new BoundedHashMap[Int, Array[MapStatus]](MAX_MAP_STATUSES, true) | ||
} | ||
|
||
/** | ||
* MapOutputTracker for the driver. This uses TimeStampedHashMap to keep track of map | ||
* output information, which allows old output information based on a TTL. | ||
*/ | ||
private[spark] class MapOutputTrackerMaster(conf: SparkConf) | ||
extends MapOutputTracker(conf) { | ||
|
||
// Cache a serialized version of the output statuses for each shuffle to send them out faster | ||
private var cacheEpoch = epoch | ||
private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]] | ||
|
||
/** | ||
* Timestamp based HashMap for storing mapStatuses in the master, so that statuses are dropped | ||
* only by explicit deregistering or by ttl-based cleaning (if set). Other than these two | ||
* scenarios, nothing should be dropped from this HashMap. | ||
*/ | ||
protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]() | ||
|
||
/** | ||
* Bounded HashMap for storing serialized statuses in the master. This allows | ||
* the HashMap stay bounded in memory-usage. Things dropped from this HashMap will be | ||
* automatically repopulated by serializing the lost statuses again . | ||
*/ | ||
protected val MAX_SERIALIZED_STATUSES = 100 | ||
private val cachedSerializedStatuses = | ||
new BoundedHashMap[Int, Array[Byte]](MAX_SERIALIZED_STATUSES, 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. I'd prefer if this just used a normal TimeStampedHashMap or even just a normal HashMap and then we added an explicit message to the MapOutputTracker to clean up the statuses when a given shuffle dependency goes out of scope. I think it's fine to relay this through the block manager for now since the driver doesn't have a direct path to the MapOutputTracker. The main motivation is to simplify this patch and avoid introducing another mechanism for deciding when to keep/evict things (this bounded size map). We can just have the invariant that if a shuffle is in scope then it's statuses are still present everywhere, if it goes out of scope it's statuses go away. If people want other behavior then this they can use the timestamped map. 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. Also I spoke with @mateiz and he was open to moving to a model where we avoid sending the entire set of statuses around: So we shouldn't over optimize for the current implementation. |
||
|
||
def registerShuffle(shuffleId: Int, numMaps: Int) { | ||
if (mapStatuses.putIfAbsent(shuffleId, new Array[MapStatus](numMaps)).isDefined) { | ||
if (mapStatuses.put(shuffleId, new Array[MapStatus](numMaps)).isDefined) { | ||
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. Any reason for this change in behaviour ? 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. I changed it to make sure that we are using map operations that are exposed by a Scala map, so that it is easier to replace TimeStampedHashMap with a Scala map later (when we have a better sense of how to clean up). In this case, since we are throwing an error if previous values exists, the behavior of put and putifAbsent is similar. Though, I realize now, that putIfAbsent is atomic, where as scala's Map.put it is not obvious whether it is atomic. 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. In this case, that is not a concern because only one thread, the DAGScheduler thread seems to be accessing this. |
||
throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice") | ||
} | ||
} | ||
|
@@ -223,6 +262,10 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) | |
} | ||
} | ||
|
||
def unregisterShuffle(shuffleId: Int) { | ||
mapStatuses.remove(shuffleId) | ||
} | ||
|
||
def incrementEpoch() { | ||
epochLock.synchronized { | ||
epoch += 1 | ||
|
@@ -259,9 +302,8 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) | |
bytes | ||
} | ||
|
||
protected override def cleanup(cleanupTime: Long) { | ||
super.cleanup(cleanupTime) | ||
cachedSerializedStatuses.clearOldValues(cleanupTime) | ||
def contains(shuffleId: Int): Boolean = { | ||
mapStatuses.contains(shuffleId) | ||
} | ||
|
||
override def stop() { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1025,6 +1025,14 @@ abstract class RDD[T: ClassTag]( | |
checkpointData.flatMap(_.getCheckpointFile) | ||
} | ||
|
||
def cleanup() { | ||
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. This does not actually cleanup the RDD, but rather just queues it for cleanup. At the very least it needs documentation, but from a user's standpoint, if they call cleanup(), they may well want it to actually block on the cleanup being completed. I can't see a particular case in which correctness relies on timely cleanup except for perf testing, though. I also don't see a particularly good reason to enqueue the cleanup here except to share the code path with the finalizer. 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. Good point. Actually I am wondering whether we should even expose this to the public interface, given that RDD.unpersist() exists (which has a field for deciding whether to block or not). However, the difference between RDD.unpersist() and this RDD.cleanup() is that the former will only unpersist the RDD, but the latter will unpersist and cleanup shuffle data if this RDD had shuffle dependencies. 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. I think it makes sense to keep cleanup() has a user-visible function rather than forcing them to rely on finalizers. In general, I suppose cleaning up an RDD means it is no longer valid to use while unpersisting just means it's no longer taking up space in the BlockManager. I can see use cases for either. 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. Why not just make it private[spark] for now? Otherwise we'll have to preserve the semantics and visibility of this function for a long time. If we change around the way we do garbage collection. But if we make it private now we can wait for feedback from people who want to use this (I'm still not sure exactly what case that would be) in deciding how and what to expose. 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. I think I agree with @pwendell . Its not clear what the semantics that we are going to guarantee. 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. Wait, why is the shuffle dependency to rdd2 not shared with rddB? In a simpler and even more extreme case, won't this result in problems for rdd2? var rdd2 = rdd1
...
rdd1.cleanup() 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. If I understand the code in CoGroupedRDD correctly, every time a CoGrouped RDD is created (join uses cogroup underneath), a new dependency object is created. So even though rddA and rddB depend on the same rdd1, they should not be sharing the shuffle dependency. Regarding the new code snippet, yes, that would result in problems for rdd2. But that can also be done currently with rdd1.unpersist(). 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. Ah, right, I'm following you on the original code snippet now. Yes, cleanup() isn't really worse than unpersist() wrt the second snippet. That tends to argue more for the removal of unpersist from the API than the inclusion of cleanup (and, yes, I realize that you are not arguing for the inclusion of cleanup in the public API.) Ideally, the automatic garbage collection introduced with this PR is sufficient to handle all of the cleanup of RDD data and metadata, and thereafter a public unpersist() or cleanup() is no more needed than is an explicit means to destroy and garbage collect Java objects. If we do want to maintain an explicit unpersist/cleanup mechanism, I think it should be behind a system administration kind of interface and protected by lots of "You're going to shoot your eye out!" warnings. 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. Hope I am missing something here ... After involing cleanup, use of rddA would result in errors - coupled with lazy execution, users would actually can end up cleaning up rdd's which have not yet been 'used'. In particular, cleanup should ensure that all pending jobs which require the rdd should have finished. Contrived example:rdd3 = rdd1.join(rdd2) rdd3.count()would cause issues for rdd3 - this is, arguably, bad code from spark dev point of view - but unlike with unpersist, where we can recover with performance panalty, with cleanup we will fail. From a (naive ?) users point of view, rdd1 has already been "used" by the time it is getting cleaned up. 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. I have not exactly tested this but I dont think it will fail. A cleaned up RDD should, for all purposes, same as a RDD that has never been submitted as part of a job, so never computed. So it will just be recomputed from scratch. Nonetheless, I agree, that cleanup should be exposed only with the propers semantics and safeguards (throw exception if reused after explicit cleanup). So for now we will not expose it. I have removed it and will updated the PR soon. |
||
logInfo("Cleanup called on RDD " + id) | ||
sc.cleaner.cleanRDD(this) | ||
dependencies.filter(_.isInstanceOf[ShuffleDependency[_, _]]) | ||
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. This is probably not important, but a while loop would be much faster here. This would likely only be a problem if the dependency list was very long or many RDDs were being cleaned up simultaneously (an ML or streaming program, perhaps). Feel free to not change this if you feel that the code change is not worth the performance benefit here. |
||
.map(_.asInstanceOf[ShuffleDependency[_, _]].shuffleId) | ||
.foreach(sc.cleaner.cleanShuffle) | ||
} | ||
|
||
// ======================================================================= | ||
// Other internal methods and fields | ||
// ======================================================================= | ||
|
@@ -1104,4 +1112,20 @@ abstract class RDD[T: ClassTag]( | |
new JavaRDD(this)(elementClassTag) | ||
} | ||
|
||
override def finalize() { | ||
try { | ||
cleanup() | ||
} catch { | ||
case t: Throwable => | ||
// Paranoia - If logError throws error as well, report to stderr. | ||
try { | ||
logError("Error in finalize", t) | ||
} catch { | ||
case _ => | ||
System.err.println("Error in finalize (and could not write to logError): " + t) | ||
} | ||
} finally { | ||
super.finalize() | ||
} | ||
} | ||
} |
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.
can we add cleared broadcast variables also in here ?
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.
Yes, I plan to do that. I am going to pull in this PR