Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
51 commits
Select commit Hold shift + click to select a range
1e752f1
Added unpersist method to Broadcast.
Feb 5, 2014
80dd977
Fix for Broadcast unpersist patch.
Feb 6, 2014
e427a9e
Added ContextCleaner to automatically clean RDDs and shuffles when th…
tdas Feb 14, 2014
8512612
Changed TimeStampedHashMap to use WrappedJavaHashMap.
tdas Feb 14, 2014
a24fefc
Merge remote-tracking branch 'apache/master' into state-cleanup
tdas Mar 11, 2014
cb0a5a6
Fixed docs and styles.
tdas Mar 11, 2014
ae9da88
Removed unncessary TimeStampedHashMap from DAGScheduler, added try-ca…
tdas Mar 12, 2014
e61daa0
Modifications based on the comments on PR 126.
tdas Mar 13, 2014
a7260d3
Added try-catch in context cleaner and null value cleaning in TimeSta…
tdas Mar 17, 2014
892b952
Removed use of BoundedHashMap, and made BlockManagerSlaveActor cleanu…
tdas Mar 18, 2014
e1fba5f
Style fix
tdas Mar 19, 2014
f2881fd
Changed ContextCleaner to use ReferenceQueue instead of finalizer
tdas Mar 25, 2014
620eca3
Changes based on PR comments.
tdas Mar 25, 2014
a007307
Merge remote-tracking branch 'apache/master' into state-cleanup
tdas Mar 25, 2014
d2f8b97
Removed duplicate unpersistRDD.
tdas Mar 25, 2014
6c9dcf6
Added missing Apache license
tdas Mar 25, 2014
c7ccef1
Merge branch 'bc-unpersist-merge' of github.com:ignatich/incubator-sp…
andrewor14 Mar 26, 2014
ba52e00
Refactor broadcast classes
andrewor14 Mar 26, 2014
d0edef3
Add framework for broadcast cleanup
andrewor14 Mar 26, 2014
544ac86
Clean up broadcast blocks through BlockManager*
andrewor14 Mar 26, 2014
e95479c
Add tests for unpersisting broadcast
andrewor14 Mar 27, 2014
f201a8d
Test broadcast cleanup in ContextCleanerSuite + remove BoundedHashMap
andrewor14 Mar 27, 2014
c92e4d9
Merge github.com:apache/spark into cleanup
andrewor14 Mar 27, 2014
0d17060
Import, comments, and style fixes (minor)
andrewor14 Mar 28, 2014
34f436f
Generalize BroadcastBlockId to remove BroadcastHelperBlockId
andrewor14 Mar 28, 2014
fbfeec8
Add functionality to query executors for their local BlockStatuses
andrewor14 Mar 29, 2014
88904a3
Make TimeStampedWeakValueHashMap a wrapper of TimeStampedHashMap
andrewor14 Mar 29, 2014
e442246
Merge github.com:apache/spark into cleanup
andrewor14 Mar 29, 2014
8557c12
Merge github.com:apache/spark into cleanup
andrewor14 Mar 30, 2014
7edbc98
Merge remote-tracking branch 'apache-github/master' into state-cleanup
tdas Mar 31, 2014
634a097
Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup
andrewor14 Mar 31, 2014
7ed72fb
Fix style test fail + remove verbose test message regarding broadcast
andrewor14 Mar 31, 2014
5016375
Address TD's comments
andrewor14 Apr 1, 2014
f0aabb1
Correct semantics for TimeStampedWeakValueHashMap + add tests
andrewor14 Apr 2, 2014
762a4d8
Merge pull request #1 from andrewor14/cleanup
tdas Apr 2, 2014
a6460d4
Merge github.com:apache/spark into cleanup
andrewor14 Apr 4, 2014
c5b1d98
Address Patrick's comments
andrewor14 Apr 4, 2014
a2cc8bc
Merge remote-tracking branch 'apache/master' into state-cleanup
tdas Apr 4, 2014
ada45f0
Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup
andrewor14 Apr 4, 2014
cd72d19
Make automatic cleanup configurable (not documented)
andrewor14 Apr 4, 2014
b27f8e8
Merge pull request #3 from andrewor14/cleanup
tdas Apr 4, 2014
a430f06
Fixed compilation errors.
tdas Apr 4, 2014
104a89a
Fixed failing BroadcastSuite unit tests by introducing blocking for r…
tdas Apr 4, 2014
6222697
Fixed bug and adding unit test for removeBroadcast in BlockManagerSuite.
tdas Apr 4, 2014
41c9ece
Added more unit tests for BlockManager, DiskBlockManager, and Context…
tdas Apr 7, 2014
2b95b5e
Added more documentation on Broadcast implementations, specially whic…
tdas Apr 7, 2014
4d05314
Scala style fix.
tdas Apr 7, 2014
cff023c
Fixed issues based on Andrew's comments.
tdas Apr 7, 2014
d25a86e
Fixed stupid typo.
tdas Apr 7, 2014
f489fdc
Merge remote-tracking branch 'apache/master' into state-cleanup
tdas Apr 8, 2014
61b8d6e
Fixed issue with Tachyon + new BlockManager methods.
tdas Apr 8, 2014
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Removed use of BoundedHashMap, and made BlockManagerSlaveActor cleanu…
…p shuffle metadata in MapOutputTrackerWorker.
  • Loading branch information
tdas committed Mar 18, 2014
commit 892b9520d828cfa7049e6ec70345b3502b139a8e
19 changes: 13 additions & 6 deletions core/src/main/scala/org/apache/spark/ContextCleaner.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark
import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}

import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
import org.apache.spark.storage.StorageLevel
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 used


/** Listener class used for testing when any item has been cleaned by the Cleaner class */
private[spark] trait CleanerListener {
Expand Down Expand Up @@ -61,19 +62,19 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
}

/**
* Clean RDD data. Do not perform any time or resource intensive
* Schedule cleanup of RDD data. Do not perform any time or resource intensive
* computation in this function as this is called from a finalize() function.
*/
def cleanRDD(rddId: Int) {
def scheduleRDDCleanup(rddId: Int) {
enqueue(CleanRDD(rddId))
logDebug("Enqueued RDD " + rddId + " for cleaning up")
}

/**
* Clean shuffle data. Do not perform any time or resource intensive
* Schedule cleanup of shuffle data. Do not perform any time or resource intensive
* computation in this function as this is called from a finalize() function.
*/
def cleanShuffle(shuffleId: Int) {
def scheduleShuffleCleanup(shuffleId: Int) {
enqueue(CleanShuffle(shuffleId))
logDebug("Enqueued shuffle " + shuffleId + " for cleaning up")
}
Expand All @@ -83,6 +84,13 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
listeners += listener
}

/** Unpersists RDD and remove all blocks for it from memory and disk. */
def unpersistRDD(rddId: Int, blocking: Boolean) {
logDebug("Unpersisted RDD " + rddId)
sc.env.blockManager.master.removeRdd(rddId, blocking)
sc.persistentRdds.remove(rddId)
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Once you rebase against master, sc.unpersist pretty much does exactly the same thing. (You might have to change the argument from an RDD to an Int though).


/**
* Enqueue a cleaning task. Do not perform any time or resource intensive
* computation in this function as this is called from a finalize() function.
Expand Down Expand Up @@ -115,8 +123,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
private def doCleanRDD(rddId: Int) {
try {
logDebug("Cleaning RDD " + rddId)
blockManagerMaster.removeRdd(rddId, false)
sc.persistentRdds.remove(rddId)
unpersistRDD(rddId, false)
listeners.foreach(_.rddCleaned(rddId))
logInfo("Cleaned RDD " + rddId)
} catch {
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/scala/org/apache/spark/Dependency.scala
Original file line number Diff line number Diff line change
Expand Up @@ -56,15 +56,15 @@ class ShuffleDependency[K, V](
override def finalize() {
try {
if (rdd != null) {
rdd.sparkContext.cleaner.cleanShuffle(shuffleId)
rdd.sparkContext.cleaner.scheduleShuffleCleanup(shuffleId)
}
} catch {
case t: Throwable =>
// Paranoia - If logError throws error as well, report to stderr.
try {
logError("Error in finalize", t)
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 _ =>
case _ : Throwable =>
System.err.println("Error in finalize (and could not write to logError): " + t)
}
} finally {
Expand Down
106 changes: 55 additions & 51 deletions core/src/main/scala/org/apache/spark/MapOutputTracker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark
import java.io._
import java.util.zip.{GZIPInputStream, GZIPOutputStream}

import scala.collection.mutable.{HashSet, Map}
import scala.collection.mutable.{HashSet, HashMap, Map}
import scala.concurrent.Await

import akka.actor._
Expand All @@ -34,6 +34,7 @@ private[spark] case class GetMapOutputStatuses(shuffleId: Int)
extends MapOutputTrackerMessage
private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage

/** Actor class for MapOutputTrackerMaster */
private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster)
extends Actor with Logging {
def receive = {
Expand All @@ -50,28 +51,35 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster
}

/**
* Class that keeps track of the location of the location of the map output of
* Class that keeps track of the location of the map 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
/** Set to the MapOutputTrackerActor living on the driver */
var trackerActor: ActorRef = _

/** 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.
/**
* 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.


// 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 = {
/** Remembers which map output locations are currently being fetched on a worker */
private val fetching = new HashSet[Int]

/**
* Send a message to the trackerActor and get its result within a default timeout, or
* throw a SparkException if this fails.
*/
protected def askTracker(message: Any): Any = {
try {
val future = trackerActor.ask(message)(timeout)
Await.result(future, timeout)
Expand All @@ -81,17 +89,17 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging
}
}

// Send a one-way message to the trackerActor, to which we expect it to reply with true.
private def communicate(message: Any) {
/** Send a one-way message to the trackerActor, to which we expect it to reply with true. */
protected def sendTracker(message: Any) {
if (askTracker(message) != true) {
throw new SparkException("Error reply received from MapOutputTracker")
}
}

// Remembers which map output locations are currently being fetched on a worker
private val fetching = new HashSet[Int]

// Called on possibly remote nodes to get the server URIs and output sizes for a given shuffle
/**
* Called from executors to get the server URIs and
* output sizes of the map outputs of a given shuffle
*/
def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = {
val statuses = mapStatuses.get(shuffleId).orNull
if (statuses == null) {
Expand Down Expand Up @@ -150,22 +158,18 @@ private[spark] abstract 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.


def stop() {
communicate(StopMapOutputTracker)
mapStatuses.clear()
trackerActor = null
}

// Called to get current epoch number
/** Called to get current epoch number */
def getEpoch: Long = {
epochLock.synchronized {
return epoch
}
}

// Called on workers to update the epoch number, potentially clearing old outputs
// because of a fetch failure. (Each worker task calls this with the latest epoch
// number on the master at the time it was created.)
/**
* Called from executors to update the epoch number, potentially clearing old outputs
* because of a fetch failure. Each worker task calls this with the latest epoch
* number on the master at the time it was created.
*/
def updateEpoch(newEpoch: Long) {
epochLock.synchronized {
if (newEpoch > epoch) {
Expand All @@ -175,24 +179,17 @@ private[spark] abstract 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) {
/** Unregister shuffle data */
def unregisterShuffle(shuffleId: Int) {
mapStatuses.remove(shuffleId)
}

/**
* 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. Its okay to
* keep the cache size small as it unlikely that there will be a very large number of
* stages active simultaneously in the worker.
*/
protected val mapStatuses = new BoundedHashMap[Int, Array[MapStatus]](
conf.getInt("spark.mapOutputTracker.cacheSize", 100), true
)
def stop() {
sendTracker(StopMapOutputTracker)
Copy link
Contributor

Choose a reason for hiding this comment

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

Does this method get called on all of the executors (in addition to the master)? If so, does the MapOutputTrackerMasterActor get many StopMapOutputTracker messages when a SparkContext shuts down?

Copy link
Contributor

Choose a reason for hiding this comment

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

Looks like this is only called when env.stop() is called, which is only called on the driver when we stop the SparkContext. Like many other things in MapOutputTracker, this should really be in MapOutputTrackerMaster, not the abstract parent. I'll move it.

mapStatuses.clear()
trackerActor = null
}
}

/**
Expand All @@ -202,7 +199,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr
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
/** Cache a serialized version of the output statuses for each shuffle to send them out faster */
private var cacheEpoch = epoch

/**
Expand All @@ -211,7 +208,6 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
* 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]]()
private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]()

Expand All @@ -232,13 +228,15 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
}
}

/** Register multiple map output information for the given shuffle */
def registerMapOutputs(shuffleId: Int, statuses: Array[MapStatus], changeEpoch: Boolean = false) {
mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses)
if (changeEpoch) {
incrementEpoch()
}
}

/** Unregister map output information of the given shuffle, mapper and block manager */
def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) {
val arrayOpt = mapStatuses.get(shuffleId)
if (arrayOpt.isDefined && arrayOpt.get != null) {
Expand All @@ -254,11 +252,17 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
}
}

def unregisterShuffle(shuffleId: Int) {
/** Unregister shuffle data */
override def unregisterShuffle(shuffleId: Int) {
mapStatuses.remove(shuffleId)
cachedSerializedStatuses.remove(shuffleId)
}

/** Check if the given shuffle is being tracked */
def containsShuffle(shuffleId: Int): Boolean = {
cachedSerializedStatuses.contains(shuffleId) || mapStatuses.contains(shuffleId)
}

def incrementEpoch() {
epochLock.synchronized {
epoch += 1
Expand Down Expand Up @@ -295,26 +299,26 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
bytes
}

def contains(shuffleId: Int): Boolean = {
cachedSerializedStatuses.contains(shuffleId) || mapStatuses.contains(shuffleId)
}

override def stop() {
super.stop()
metadataCleaner.cancel()
cachedSerializedStatuses.clear()
}

override def updateEpoch(newEpoch: Long) {
// This might be called on the MapOutputTrackerMaster if we're running in local mode.
}

protected def cleanup(cleanupTime: Long) {
mapStatuses.clearOldValues(cleanupTime)
cachedSerializedStatuses.clearOldValues(cleanupTime)
}
Copy link
Contributor

Choose a reason for hiding this comment

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

this should be private

}

Copy link
Contributor

Choose a reason for hiding this comment

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

Not related to your patch, but looks like we over-indented in convertMapStatuses in L346, and GitHub won't let me comment down there.

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.

/**
* MapOutputTracker for the workers, which fetches map output information from the driver's
* MapOutputTrackerMaster.
*/
private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@pwendell Given that MapOutputTrackerWorker only change the type of HashMap used over MapOutputTracker, does it make sense to define two separate MapOutputTrackerMaster and MapOutputTrackerWorker? Or keep it as before, MapOutputTracker (for workers) and MapOutputTrackerMaster (for driver)? I think since we instantiate different classes in the driver and the worker, its best to name them accordingly.

protected val mapStatuses = new HashMap[Int, Array[MapStatus]]
}

private[spark] object MapOutputTracker {
private val LOG_BASE = 1.1

Expand Down
25 changes: 13 additions & 12 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -165,18 +165,6 @@ object SparkEnv extends Logging {
}
}

val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
"BlockManagerMaster",
new BlockManagerMasterActor(isLocal, conf)), conf)
val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster,
serializer, conf, securityManager)

val connectionManager = blockManager.connectionManager

val broadcastManager = new BroadcastManager(isDriver, conf, securityManager)

val cacheManager = new CacheManager(blockManager)

// Have to assign trackerActor after initialization as MapOutputTrackerActor
// requires the MapOutputTracker itself
val mapOutputTracker = if (isDriver) {
Expand All @@ -188,6 +176,19 @@ object SparkEnv extends Logging {
"MapOutputTracker",
new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]))

val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
"BlockManagerMaster",
new BlockManagerMasterActor(isLocal, conf)), conf)

val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster,
serializer, conf, securityManager, mapOutputTracker)

val connectionManager = blockManager.connectionManager

val broadcastManager = new BroadcastManager(isDriver, conf, securityManager)

val cacheManager = new CacheManager(blockManager)

val shuffleFetcher = instantiateClass[ShuffleFetcher](
"spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher")

Expand Down
15 changes: 3 additions & 12 deletions core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -165,8 +165,7 @@ abstract class RDD[T: ClassTag](
*/
def unpersist(blocking: Boolean = true): RDD[T] = {
logInfo("Removing RDD " + id + " from persistence list")
sc.env.blockManager.master.removeRdd(id, blocking)
sc.persistentRdds.remove(id)
sc.cleaner.unpersistRDD(id, blocking)
storageLevel = StorageLevel.NONE
this
}
Expand Down Expand Up @@ -1025,14 +1024,6 @@ abstract class RDD[T: ClassTag](
checkpointData.flatMap(_.getCheckpointFile)
}

def cleanup() {
logInfo("Cleanup called on RDD " + id)
sc.cleaner.cleanRDD(id)
dependencies.filter(_.isInstanceOf[ShuffleDependency[_, _]])
.map(_.asInstanceOf[ShuffleDependency[_, _]].shuffleId)
.foreach(sc.cleaner.cleanShuffle)
}

// =======================================================================
// Other internal methods and fields
// =======================================================================
Expand Down Expand Up @@ -1114,14 +1105,14 @@ abstract class RDD[T: ClassTag](

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