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
Added more unit tests for BlockManager, DiskBlockManager, and Context…
…Cleaner.
  • Loading branch information
tdas committed Apr 7, 2014
commit 41c9ecec21e61bcc077d6e3ea052a3b7a2d4b01a
12 changes: 9 additions & 3 deletions core/src/main/scala/org/apache/spark/ContextCleaner.scala
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,9 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {

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

/** Whether the cleaning thread will block on cleanup tasks */
private val blockOnCleanupTasks = sc.conf.getBoolean("spark.cleaner.referenceTracking.blocking", false)

@volatile private var stopped = false

/** Attach a listener object to get information of when objects are cleaned. */
Expand Down Expand Up @@ -112,9 +115,12 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
logDebug("Got cleaning task " + task)
referenceBuffer -= reference.get
task match {
case CleanRDD(rddId) => doCleanupRDD(rddId, blocking = false)
case CleanShuffle(shuffleId) => doCleanupShuffle(shuffleId, blocking = false)
case CleanBroadcast(broadcastId) => doCleanupBroadcast(broadcastId, blocking = false)
case CleanRDD(rddId) =>
doCleanupRDD(rddId, blocking = blockOnCleanupTasks)
case CleanShuffle(shuffleId) =>
doCleanupShuffle(shuffleId, blocking = blockOnCleanupTasks)
case CleanBroadcast(broadcastId) =>
doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks)
}
}
} catch {
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -229,7 +229,7 @@ class SparkContext(
dagScheduler.start()

private[spark] val cleaner: Option[ContextCleaner] =
if (conf.getBoolean("spark.cleaner.automatic", true)) {
if (conf.getBoolean("spark.cleaner.referenceTracking", true)) {
Some(new ContextCleaner(this))
} else None
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 style guide says this None needs its own line w/ brackets.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Right. Fixed. Will push with all updates.


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,15 @@ private[spark] class BlockManager(
}
}

/**
* Get the ids of existing blocks that match the given filter. Note that this will
* query the blocks stored in the disk block manager (that the block manager
* may not know of).
*/
def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = {
(blockInfo.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq
}

/**
* Tell the master about the current storage status of a block. This will send a block update
* message reflecting the current status, *not* the desired storage level in its block info.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log

/** Remove all blocks belonging to the given broadcast. */
def removeBroadcast(broadcastId: Long, removeFromMaster: Boolean, blocking: Boolean) {
val future = askDriverWithReply[Future[Seq[Int]]](RemoveBroadcast(broadcastId, removeFromMaster))
val future = askDriverWithReply[Future[Seq[Int]]](
RemoveBroadcast(broadcastId, removeFromMaster))
future.onFailure {
case e: Throwable =>
logError("Failed to remove broadcast " + broadcastId +
Expand All @@ -156,8 +157,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log
}

/**
* Return the block's status on all block managers, if any. This can potentially be an
* expensive operation and is used mainly for testing.
* Return the block's status on all block managers, if any. NOTE: This is a
* potentially expensive operation and should only be used for testing.
*
* If askSlaves is true, this invokes the master to query each block manager for the most
* updated block statuses. This is useful when the master is not informed of the given block
Expand All @@ -184,6 +185,22 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log
}.toMap
}

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 introduced for testing broadcast cleanup (but can be used for other things in the future).

/**
* Return a list of ids of existing blocks such that the ids match the given filter. NOTE: This
* is a potentially expensive operation and should only be used for testing.
*
* If askSlaves is true, this invokes the master to query each block manager for the most
* updated block statuses. This is useful when the master is not informed of the given block
* by all block managers.
*/
def getMatcinghBlockIds(
filter: BlockId => Boolean,
askSlaves: Boolean): Seq[BlockId] = {
val msg = GetMatchingBlockIds(filter, askSlaves)
val future = askDriverWithReply[Future[Seq[BlockId]]](msg)
Await.result(future, timeout)
}

/** Stop the driver actor, called only on the Spark driver node */
def stop() {
if (driverActor != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,9 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
case GetBlockStatus(blockId, askSlaves) =>
sender ! blockStatus(blockId, askSlaves)

case GetMatchingBlockIds(filter, askSlaves) =>
sender ! getMatchingBlockIds(filter, askSlaves)

case RemoveRdd(rddId) =>
sender ! removeRdd(rddId)

Expand Down Expand Up @@ -266,8 +269,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
}

/**
* Return the block's status for all block managers, if any. This can potentially be an
* expensive operation and is used mainly for testing.
* Return the block's status for all block managers, if any. NOTE: This is a
* potentially expensive operation and should only be used for testing.
*
* If askSlaves is true, the master queries each block manager for the most updated block
* statuses. This is useful when the master is not informed of the given block by all block
Expand All @@ -294,6 +297,32 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
}.toMap
}

/**
* Return the ids of blocks present in all the block managers that match the given filter.
* NOTE: This is a potentially expensive operation and should only be used for testing.
*
* If askSlaves is true, the master queries each block manager for the most updated block
* statuses. This is useful when the master is not informed of the given block by all block
* managers.
*/
private def getMatchingBlockIds(
filter: BlockId => Boolean,
askSlaves: Boolean): Future[Seq[BlockId]] = {
import context.dispatcher
val getMatchingBlockIds = GetMatchingBlockIds(filter)
Future.sequence(
blockManagerInfo.values.map { info =>
val future =
if (askSlaves) {
info.slaveActor.ask(getMatchingBlockIds)(akkaTimeout).mapTo[Seq[BlockId]]
} else {
Future { info.blocks.keys.filter(filter).toSeq }
}
future
}
).map(_.flatten.toSeq)
}

private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) {
if (!blockManagerInfo.contains(id)) {
blockManagerIdByExecutor.get(id.executorId) match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,5 +115,8 @@ private[storage] object BlockManagerMessages {
case class GetBlockStatus(blockId: BlockId, askSlaves: Boolean = true)
extends ToBlockManagerMaster

case class GetMatchingBlockIds(filter: BlockId => Boolean, askSlaves: Boolean = true)
extends ToBlockManagerMaster

case object ExpireDeadHosts extends ToBlockManagerMaster
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,28 +39,34 @@ class BlockManagerSlaveActor(
// Operations that involve removing blocks may be slow and should be done asynchronously
override def receive = {
case RemoveBlock(blockId) =>
doAsync("removing block", sender) {
doAsync[Boolean]("removing block", sender) {
blockManager.removeBlock(blockId)
true
}

case RemoveRdd(rddId) =>
doAsync("removing RDD", sender) {
doAsync[Int]("removing RDD", sender) {
blockManager.removeRdd(rddId)
}

case RemoveShuffle(shuffleId) =>
doAsync("removing shuffle", sender) {
doAsync[Boolean]("removing shuffle", sender) {
if (mapOutputTracker != null) {
mapOutputTracker.unregisterShuffle(shuffleId)
}
blockManager.shuffleBlockManager.removeShuffle(shuffleId)
}

case RemoveBroadcast(broadcastId, tellMaster) =>
doAsync("removing RDD", sender) {
doAsync[Int]("removing RDD", sender) {
blockManager.removeBroadcast(broadcastId, tellMaster)
}

case GetBlockStatus(blockId, _) =>
sender ! blockManager.getStatus(blockId)

case GetMatchingBlockIds(filter, _) =>
sender ! blockManager.getMatchingBlockIds(filter)
}

private def doAsync[T](actionMessage: String, responseActor: ActorRef)(body: => T) {
Expand All @@ -70,7 +76,7 @@ class BlockManagerSlaveActor(
response
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 rename body to response in the first place?

}
future.onSuccess { case response =>
logDebug("Successful in " + actionMessage + ", response is " + response)
logDebug("Done " + actionMessage + ", response is " + response)
Copy link
Contributor

Choose a reason for hiding this comment

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

We probably want to include the RDD/shuffle/broadcast ID in the action message

responseActor ! response
logDebug("Sent response: " + response + " to " + responseActor)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD
private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
private var shuffleSender : ShuffleSender = null


Copy link
Contributor

Choose a reason for hiding this comment

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

nit: this was probably not intended

addShutdownHook()

/**
Expand Down Expand Up @@ -95,6 +96,15 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD
getBlockLocation(blockId).file.exists()
}

/** List all the blocks currently stored in disk by the disk manager. */
Copy link
Contributor

Choose a reason for hiding this comment

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

typically people say "stored on disk" not "stored in disk"

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Cool.

def getAllBlocks(): Seq[BlockId] = {
// Get all the files inside the array of array of directories
subDirs.flatten.filter(_ != null).flatMap { dir =>
val files = dir.list()
if (files != null) files else Seq.empty
}.map(BlockId.apply)
}

/** Produces a unique block id and File suitable for intermediate results. */
def createTempBlock(): (TempBlockId, File) = {
var blockId = new TempBlockId(UUID.randomUUID())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -171,8 +171,11 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging {

/** Remove all the blocks / files and metadata related to a particular shuffle. */
def removeShuffle(shuffleId: ShuffleId): Boolean = {
// Do not change the ordering of this, if shuffleStates should be removed only
// after the corresponding shuffle blocks have been removed
val cleaned = removeShuffleBlocks(shuffleId)
shuffleStates.remove(shuffleId)
removeShuffleBlocks(shuffleId)
cleaned
}

/** Remove all the blocks / files related to a particular shuffle. */
Expand Down
12 changes: 6 additions & 6 deletions core/src/test/scala/org/apache/spark/BroadcastSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext {
// Verify that the broadcast file is created, and blocks are persisted only on the driver
def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) {
assert(blockIds.size === 1)
val statuses = bmm.getBlockStatus(blockIds.head)
val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true)
assert(statuses.size === 1)
statuses.head match { case (bm, status) =>
assert(bm.executorId === "<driver>", "Block should only be on the driver")
Expand All @@ -142,7 +142,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext {
// Verify that blocks are persisted in both the executors and the driver
def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) {
assert(blockIds.size === 1)
val statuses = bmm.getBlockStatus(blockIds.head)
val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true)
assert(statuses.size === numSlaves + 1)
statuses.foreach { case (_, status) =>
assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK)
Expand All @@ -155,7 +155,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext {
// is true. In the latter case, also verify that the broadcast file is deleted on the driver.
def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) {
assert(blockIds.size === 1)
val statuses = bmm.getBlockStatus(blockIds.head)
val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true)
val expectedNumBlocks = if (removeFromDriver) 0 else 1
val possiblyNot = if (removeFromDriver) "" else " not"
assert(statuses.size === expectedNumBlocks,
Expand Down Expand Up @@ -197,7 +197,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext {
// Verify that blocks are persisted only on the driver
def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) {
blockIds.foreach { blockId =>
val statuses = bmm.getBlockStatus(blockIds.head)
val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true)
assert(statuses.size === 1)
statuses.head match { case (bm, status) =>
assert(bm.executorId === "<driver>", "Block should only be on the driver")
Expand All @@ -211,7 +211,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext {
// Verify that blocks are persisted in both the executors and the driver
def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) {
blockIds.foreach { blockId =>
val statuses = bmm.getBlockStatus(blockId)
val statuses = bmm.getBlockStatus(blockId, askSlaves = true)
if (blockId.field == "meta") {
// Meta data is only on the driver
assert(statuses.size === 1)
Expand All @@ -235,7 +235,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext {
val expectedNumBlocks = if (removeFromDriver) 0 else 1
val possiblyNot = if (removeFromDriver) "" else " not"
blockIds.foreach { blockId =>
val statuses = bmm.getBlockStatus(blockId)
val statuses = bmm.getBlockStatus(blockId, askSlaves = true)
assert(statuses.size === expectedNumBlocks,
"Block should%s be unpersisted on the driver".format(possiblyNot))
}
Expand Down
Loading