Skip to content
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

Closed
wants to merge 51 commits into from
Closed
Show file tree
Hide file tree
Changes from 5 commits
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
126 changes: 126 additions & 0 deletions core/src/main/scala/org/apache/spark/ContextCleaner.scala
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)
Copy link
Contributor

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 ?

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 plan to do that. I am going to pull in this PR

}

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

Choose a reason for hiding this comment

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

Should this be synchronized?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

BlockingQueue implementations are thread-safe. :)

Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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)
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.start()
}

/** Stop the cleaner */
def stop() {
synchronized { 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.

}

/** Clean (unpersist) RDD data. */
def cleanRDD(rdd: RDD[_]) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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?
What I can do is write in the function docs that do not do any time and resource consuming stuff in this.

Copy link
Contributor

Choose a reason for hiding this comment

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

Choose a reason for hiding this comment

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

Choose a reason for hiding this comment

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

We should make this configurable

Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, @aarondav answered it for me.

Copy link
Contributor

Choose a reason for hiding this comment

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

Choose a reason for hiding this comment

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

What do you think about this?

for (task <- taskOpt) {
   blah blah blah
}

could also include a few monads and higher order functions while we're at it, though

Copy link
Contributor

Choose a reason for hiding this comment

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

Or Option(...).foreach { ... }, which I prefer since it doesn't use any monads

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 =>
Copy link
Contributor

Choose a reason for hiding this comment

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

Choose a reason for hiding this comment

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

wrap the body in curly braces

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

latter

}
}

/** Perform RDD cleaning */
private def doCleanRDD(sc: SparkContext, rddId: Int) {
Copy link
Contributor

Choose a reason for hiding this comment

The 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 ContextCleaner? You could just add a val:

private[spark] class ContextCleaner(val sc: SparkContext) extends Logging {

Otherwise it seems a little odd here to pass a SparkContext again.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Already fixed that. Left over code. Dont need val sc for that.

logDebug("Cleaning rdd " + rddId)
Copy link
Contributor

Choose a reason for hiding this comment

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

Why doesn't this just call rdd.unpersist()? It seems we have duplicated code paths here.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

what about if the RDD unpersist function called sc.cleaner.doCleanRDD(id)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Choose a reason for hiding this comment

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

It seems that if removeRdd throws an exception, then that can crash your cleaning thread, since keepCleaning only catches InterruptedException. Either add a try catch here and in doCleanShuffle, or within the while loop of keepCleaning.

Copy link
Contributor

Choose a reason for hiding this comment

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

Choose a reason for hiding this comment

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

Minor: looks like you only ever use this with blockManager.master, so you might as well define this as env.blockManager.master


private def isStopped = synchronized { stopped }
}
21 changes: 20 additions & 1 deletion core/src/main/scala/org/apache/spark/Dependency.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Copy link
Member

Choose a reason for hiding this comment

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

@tdas Hey TD, A try catch on logging ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

The 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()
}
}
}


Expand Down
72 changes: 57 additions & 15 deletions core/src/main/scala/org/apache/spark/MapOutputTracker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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}

Expand All @@ -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)
Expand All @@ -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.
Copy link
Contributor

Choose a reason for hiding this comment

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

"... keeps track of the location of the location of the mapt output..."

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Choose a reason for hiding this comment

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

Choose a reason for hiding this comment

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

nit: not part of your patch, but can this just be AnyRef?

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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 = {
Expand Down Expand Up @@ -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))
}
Expand All @@ -150,13 +156,12 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging {
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Not exactly related to your patch, but does the MOTMaster ever call getServerStatus? It seems unnecessary for the master since it already has all the data. Now that we also have a MOTWorker we should put this and other methods / fields in there (askTracker, communicate, val fetching, getServerStatus and updateEpoch). Right now it's a little confusing for MOTMaster to be able to ask himself for status.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Makes sense. Will do.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Expand All @@ -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
Copy link
Contributor

Choose a reason for hiding this comment

The 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)
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 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.

Copy link
Contributor

Choose a reason for hiding this comment

The 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:
https://spark-project.atlassian.net/browse/SPARK-1239

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

Choose a reason for hiding this comment

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

Any reason for this change in behaviour ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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")
}
}
Expand Down Expand Up @@ -223,6 +262,10 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
}
}

def unregisterShuffle(shuffleId: Int) {
mapStatuses.remove(shuffleId)
}

def incrementEpoch() {
epochLock.synchronized {
epoch += 1
Expand Down Expand Up @@ -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() {
Expand Down
8 changes: 6 additions & 2 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me
import org.apache.spark.scheduler.local.LocalBackend
import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils}
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils}
import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils}

/**
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
Expand Down Expand Up @@ -147,7 +147,7 @@ class SparkContext(
private[spark] val addedJars = HashMap[String, Long]()

// Keeps track of all persisted RDDs
private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]
private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]]
private[spark] val metadataCleaner =
new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf)

Expand Down Expand Up @@ -206,6 +206,9 @@ class SparkContext(
@volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler)
dagScheduler.start()

private[spark] val cleaner = new ContextCleaner(env)
cleaner.start()

ui.start()

/** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
Expand Down Expand Up @@ -792,6 +795,7 @@ class SparkContext(
dagScheduler = null
if (dagSchedulerCopy != null) {
metadataCleaner.cancel()
cleaner.stop()
dagSchedulerCopy.stop()
taskScheduler = null
// TODO: Cache.stop()?
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,7 @@ object SparkEnv extends Logging {
val mapOutputTracker = if (isDriver) {
new MapOutputTrackerMaster(conf)
} else {
new MapOutputTracker(conf)
new MapOutputTrackerWorker(conf)
}
mapOutputTracker.trackerActor = registerOrLookup(
"MapOutputTracker",
Expand Down
24 changes: 24 additions & 0 deletions core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1025,6 +1025,14 @@ abstract class RDD[T: ClassTag](
checkpointData.flatMap(_.getCheckpointFile)
}

def cleanup() {
Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

The 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()

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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().

Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

The 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'.
We should defer exposing this api until we have more clarity on this.

In particular, cleanup should ensure that all pending jobs which require the rdd should have finished.

Contrived example:

rdd3 = rdd1.join(rdd2)
rdd1.count()
rdd1.cleanup()
...

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.
Best option would be if we can throw an exception when cleanup is invoked in cases like this; and same exception being thrown if rdd is subsequently used (for any op) after a cleanup : this latter might be more an involved change though.

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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[_, _]])
Copy link
Contributor

Choose a reason for hiding this comment

The 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
// =======================================================================
Expand Down Expand Up @@ -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()
}
}
}
Loading