-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-1103] [WIP] Automatic garbage collection of RDD, shuffle and broadcast data #126
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
Changes from 1 commit
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
…emoveShuffle and removeBroadcast in BlockManager*
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,6 +29,6 @@ import org.apache.spark.SparkConf | |
| trait BroadcastFactory { | ||
| def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) | ||
|
Contributor
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 looks like the only change here was to remove three type signatures. Maybe this was inadvertent from the merge? These should probably all be added back.
Contributor
Author
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. Okay. But I am also adding |
||
| def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] | ||
| def unbroadcast(id: Long, removeFromDriver: Boolean) | ||
| def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) | ||
| def stop() | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -100,12 +100,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus | |
| sender ! removeRdd(rddId) | ||
|
|
||
| case RemoveShuffle(shuffleId) => | ||
| removeShuffle(shuffleId) | ||
| sender ! true | ||
| sender ! removeShuffle(shuffleId) | ||
|
|
||
| case RemoveBroadcast(broadcastId, removeFromDriver) => | ||
| removeBroadcast(broadcastId, removeFromDriver) | ||
| sender ! true | ||
| sender ! removeBroadcast(broadcastId, removeFromDriver) | ||
|
|
||
| case RemoveBlock(blockId) => | ||
| removeBlockFromWorkers(blockId) | ||
|
|
@@ -150,28 +148,41 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus | |
| // The dispatcher is used as an implicit argument into the Future sequence construction. | ||
| import context.dispatcher | ||
| val removeMsg = RemoveRdd(rddId) | ||
| Future.sequence(blockManagerInfo.values.map { bm => | ||
| bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] | ||
| }.toSeq) | ||
| Future.sequence( | ||
| blockManagerInfo.values.map { bm => | ||
| bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] | ||
| }.toSeq | ||
| ) | ||
| } | ||
|
|
||
| private def removeShuffle(shuffleId: Int) { | ||
| private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = { | ||
| // Nothing to do in the BlockManagerMasterActor data structures | ||
| import context.dispatcher | ||
| val removeMsg = RemoveShuffle(shuffleId) | ||
| blockManagerInfo.values.foreach { bm => bm.slaveActor ! removeMsg } | ||
| Future.sequence( | ||
| blockManagerInfo.values.map { bm => | ||
| bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Boolean] | ||
| }.toSeq | ||
| ) | ||
| } | ||
|
|
||
| /** | ||
| * Delegate RemoveBroadcast messages to each BlockManager because the master may not notified | ||
| * of all broadcast blocks. If removeFromDriver is false, broadcast blocks are only removed | ||
|
Contributor
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. Same question here - curious when it is and is not notified. It's probably fine to just braodcast this (no pun intended) to all block managers anyways, but I'm just curious. |
||
| * from the executors, but not from the driver. | ||
| */ | ||
| private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean) { | ||
| private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean): Future[Seq[Int]] = { | ||
| // TODO: Consolidate usages of <driver> | ||
| import context.dispatcher | ||
| val removeMsg = RemoveBroadcast(broadcastId, removeFromDriver) | ||
| blockManagerInfo.values | ||
| .filter { info => removeFromDriver || info.blockManagerId.executorId != "<driver>" } | ||
| .foreach { bm => bm.slaveActor ! removeMsg } | ||
| val requiredBlockManagers = blockManagerInfo.values.filter { info => | ||
| removeFromDriver || info.blockManagerId.executorId != "<driver>" | ||
| } | ||
|
Contributor
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 can be foreach |
||
| Future.sequence( | ||
| requiredBlockManagers.map { bm => | ||
| bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] | ||
| }.toSeq | ||
| ) | ||
| } | ||
|
|
||
| private def removeBlockManager(blockManagerId: BlockManagerId) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,7 +19,7 @@ package org.apache.spark.storage | |
|
|
||
| import scala.concurrent.Future | ||
|
|
||
| import akka.actor.Actor | ||
| import akka.actor.{ActorRef, Actor} | ||
|
|
||
| import org.apache.spark.{Logging, MapOutputTracker} | ||
| import org.apache.spark.storage.BlockManagerMessages._ | ||
|
|
@@ -39,35 +39,44 @@ class BlockManagerSlaveActor( | |
| // Operations that involve removing blocks may be slow and should be done asynchronously | ||
| override def receive = { | ||
| case RemoveBlock(blockId) => | ||
| val removeBlock = Future { blockManager.removeBlock(blockId) } | ||
| removeBlock.onFailure { case t: Throwable => | ||
| logError("Error in removing block " + blockId, t) | ||
| doAsync("removing block", sender) { | ||
| blockManager.removeBlock(blockId) | ||
| true | ||
| } | ||
|
|
||
| case RemoveRdd(rddId) => | ||
| val removeRdd = Future { sender ! blockManager.removeRdd(rddId) } | ||
| removeRdd.onFailure { case t: Throwable => | ||
| logError("Error in removing RDD " + rddId, t) | ||
| doAsync("removing RDD", sender) { | ||
| blockManager.removeRdd(rddId) | ||
| } | ||
|
|
||
| case RemoveShuffle(shuffleId) => | ||
| val removeShuffle = Future { | ||
| doAsync("removing shuffle", sender) { | ||
| blockManager.shuffleBlockManager.removeShuffle(shuffleId) | ||
| if (mapOutputTracker != null) { | ||
| mapOutputTracker.unregisterShuffle(shuffleId) | ||
| } | ||
| } | ||
| removeShuffle.onFailure { case t: Throwable => | ||
| logError("Error in removing shuffle " + shuffleId, t) | ||
| } | ||
|
|
||
| case RemoveBroadcast(broadcastId, tellMaster) => | ||
| val removeBroadcast = Future { blockManager.removeBroadcast(broadcastId, tellMaster) } | ||
| removeBroadcast.onFailure { case t: Throwable => | ||
| logError("Error in removing broadcast " + broadcastId, t) | ||
| doAsync("removing RDD", sender) { | ||
| blockManager.removeBroadcast(broadcastId, tellMaster) | ||
| } | ||
|
|
||
| case GetBlockStatus(blockId, _) => | ||
| sender ! blockManager.getStatus(blockId) | ||
| } | ||
|
|
||
| private def doAsync[T](actionMessage: String, responseActor: ActorRef)(body: => T) { | ||
| val future = Future { | ||
| logDebug(actionMessage) | ||
| val response = body | ||
| response | ||
|
Contributor
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 rename |
||
| } | ||
| future.onSuccess { case response => | ||
| logDebug("Successful in " + actionMessage + ", response is " + response) | ||
| responseActor ! response | ||
| logDebug("Sent response: " + response + " to " + responseActor) | ||
| } | ||
| future.onFailure { case t: Throwable => | ||
| logError("Error in " + actionMessage, t) | ||
| responseActor ! null.asInstanceOf[T] | ||
| } | ||
| } | ||
| } | ||
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.
I don't get why these need to be exposed (each of these is only called once). Why not just call
doCleanupetc with the appropriate arguments? We can make thoseprivate[spark]if necessary for testing.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.
Fixed. Dont really need private[spark] as the whole class is private[spark]. So just making them public is sufficient.
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.
Will push with all updates.