-
Notifications
You must be signed in to change notification settings - Fork 29.2k
[SPARK-20732][CORE] Decommission cache blocks to other executors when an executor is decommissioned #27864
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-20732][CORE] Decommission cache blocks to other executors when an executor is decommissioned #27864
Changes from 3 commits
ab0e38c
6a47615
622e1ba
d792092
b9906c2
076dd67
d12dbff
4c67660
f6b4f7c
9c6bdb6
bb324f9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -54,6 +54,7 @@ import org.apache.spark.rpc.RpcEnv | |
| import org.apache.spark.scheduler.ExecutorCacheTaskLocation | ||
| import org.apache.spark.serializer.{SerializerInstance, SerializerManager} | ||
| import org.apache.spark.shuffle.{ShuffleManager, ShuffleWriteMetricsReporter} | ||
| import org.apache.spark.storage.BlockManagerMessages.ReplicateBlock | ||
| import org.apache.spark.storage.memory._ | ||
| import org.apache.spark.unsafe.Platform | ||
| import org.apache.spark.util._ | ||
|
|
@@ -241,6 +242,9 @@ private[spark] class BlockManager( | |
|
|
||
| private var blockReplicationPolicy: BlockReplicationPolicy = _ | ||
|
|
||
| private var blockManagerDecommissioning: Boolean = false | ||
| private var decommissionManager: Option[BlockManagerDecommissionManager] = None | ||
|
|
||
| // A DownloadFileManager used to track all the files of remote blocks which are above the | ||
| // specified memory threshold. Files will be deleted automatically based on weak reference. | ||
| // Exposed for test | ||
|
|
@@ -1281,6 +1285,9 @@ private[spark] class BlockManager( | |
|
|
||
| require(blockId != null, "BlockId is null") | ||
| require(level != null && level.isValid, "StorageLevel is null or invalid") | ||
| if (blockManagerDecommissioning && blockId.isRDD) { | ||
|
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. I don't think we should reject the block?
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. Added overall design to explain why I went ahead with this - #27864 (comment) .
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. Still not sure about this part of the design. I'd like to see a test to make sure this won't cause task failures.
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. @holdenk Fixed this. Removed the code for not accepting new RDD cache blocks on Decommissioning executor. Also added a test to show that ongoing tasks which are going to cache data in future succeeds as part of BlockManagerDecommissionSuite. |
||
| throw new RDDBlockSavedOnDecommissionedBlockManagerException(blockId.asRDDId.get) | ||
| } | ||
|
|
||
| val putBlockInfo = { | ||
| val newInfo = new BlockInfo(level, classTag, tellMaster) | ||
|
|
@@ -1560,26 +1567,30 @@ private[spark] class BlockManager( | |
| def replicateBlock( | ||
| blockId: BlockId, | ||
| existingReplicas: Set[BlockManagerId], | ||
| maxReplicas: Int): Unit = { | ||
| maxReplicas: Int, | ||
| forceFetch: Boolean = true, | ||
|
prakharjain09 marked this conversation as resolved.
Outdated
|
||
| maxReplicationFailures: Option[Int] = None): Boolean = { | ||
| var replicatedSuccessfully = true | ||
|
prakharjain09 marked this conversation as resolved.
Outdated
|
||
| logInfo(s"Using $blockManagerId to pro-actively replicate $blockId") | ||
| blockInfoManager.lockForReading(blockId).foreach { info => | ||
| replicatedSuccessfully = false | ||
| val data = doGetLocalBytes(blockId, info) | ||
| val storageLevel = StorageLevel( | ||
| useDisk = info.level.useDisk, | ||
| useMemory = info.level.useMemory, | ||
| useOffHeap = info.level.useOffHeap, | ||
| deserialized = info.level.deserialized, | ||
| replication = maxReplicas) | ||
| // we know we are called as a result of an executor removal, so we refresh peer cache | ||
| // this way, we won't try to replicate to a missing executor with a stale reference | ||
| getPeers(forceFetch = true) | ||
| getPeers(forceFetch) | ||
| try { | ||
| replicate(blockId, data, storageLevel, info.classTag, existingReplicas) | ||
| replicatedSuccessfully = replicate( | ||
| blockId, data, storageLevel, info.classTag, existingReplicas, maxReplicationFailures) | ||
| } finally { | ||
| logDebug(s"Releasing lock for $blockId") | ||
| releaseLockAndDispose(blockId, data) | ||
| } | ||
| } | ||
| replicatedSuccessfully | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -1591,9 +1602,11 @@ private[spark] class BlockManager( | |
| data: BlockData, | ||
| level: StorageLevel, | ||
| classTag: ClassTag[_], | ||
| existingReplicas: Set[BlockManagerId] = Set.empty): Unit = { | ||
| existingReplicas: Set[BlockManagerId] = Set.empty, | ||
| maxReplicationFailures: Option[Int] = None): Boolean = { | ||
|
|
||
| val maxReplicationFailures = conf.get(config.STORAGE_MAX_REPLICATION_FAILURE) | ||
| val maxReplicationFailureCount = maxReplicationFailures.getOrElse( | ||
| conf.get(config.STORAGE_MAX_REPLICATION_FAILURE)) | ||
| val tLevel = StorageLevel( | ||
| useDisk = level.useDisk, | ||
| useMemory = level.useMemory, | ||
|
|
@@ -1617,7 +1630,7 @@ private[spark] class BlockManager( | |
| blockId, | ||
| numPeersToReplicateTo) | ||
|
|
||
| while(numFailures <= maxReplicationFailures && | ||
| while(numFailures <= maxReplicationFailureCount && | ||
| !peersForReplication.isEmpty && | ||
| peersReplicatedTo.size < numPeersToReplicateTo) { | ||
| val peer = peersForReplication.head | ||
|
|
@@ -1665,9 +1678,11 @@ private[spark] class BlockManager( | |
| if (peersReplicatedTo.size < numPeersToReplicateTo) { | ||
| logWarning(s"Block $blockId replicated to only " + | ||
| s"${peersReplicatedTo.size} peer(s) instead of $numPeersToReplicateTo peers") | ||
| return false | ||
| } | ||
|
|
||
| logDebug(s"block $blockId replicated to ${peersReplicatedTo.mkString(", ")}") | ||
| logInfo(s"block $blockId replicated to ${peersReplicatedTo.mkString(", ")}") | ||
|
prakharjain09 marked this conversation as resolved.
Outdated
|
||
| return true | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -1761,6 +1776,57 @@ private[spark] class BlockManager( | |
| blocksToRemove.size | ||
| } | ||
|
|
||
| def decommissionBlockManager(): Unit = { | ||
| if (!blockManagerDecommissioning) { | ||
|
prakharjain09 marked this conversation as resolved.
|
||
| logInfo("Starting block manager decommissioning process") | ||
| blockManagerDecommissioning = true | ||
| decommissionManager = Some(new BlockManagerDecommissionManager) | ||
|
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. I guess I'm not sure why were doing this in this manner? It seems like if we did this as a blocking call and returned true when we were done we could move on to a next phase of decommissioning once the blocks were replicated in the driver?
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. @holdenk I have added the overall design for this change. Please review the same and provide your suggestions/feedback. |
||
| decommissionManager.foreach(_.start()) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers | ||
| * Visible for testing | ||
| */ | ||
| def offloadRddCacheBlocks(): Unit = { | ||
| val replicateBlocksInfo = master.getReplicateInfoForRDDBlocks(blockManagerId) | ||
|
|
||
| if (replicateBlocksInfo.nonEmpty) { | ||
| logInfo(s"Need to replicate ${replicateBlocksInfo.size} blocks " + | ||
| s"for block manager decommissioning") | ||
|
prakharjain09 marked this conversation as resolved.
Outdated
|
||
| // Refresh peer list once before starting replication | ||
| getPeers(true) | ||
| } | ||
|
|
||
| // Maximum number of storage replication failure which replicateBlock can handle | ||
| // before giving up for one block | ||
|
Member
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.
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.
|
||
| val maxReplicationFailures = conf.get( | ||
| config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE) | ||
|
|
||
| val blocksFailedReplication = replicateBlocksInfo.filterNot { | ||
| case ReplicateBlock(blockId, existingReplicas, maxReplicas) => | ||
| val replicatedSuccessfully = replicateBlock( | ||
|
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. I don't know if we need it but replicateBlock is blocking and it seems like maybe async + futures might help us migrate more blocks? Especially if one host is underload we might block on sending a block to that host before we move forward.
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. @holdenk I am not sure how this will behave when multiple executors on same host machine are decommissioning. And each one of them is doing it in parallel - may cause some sort of network congestion? I have updated code to do replication in ThreadPool of size 4. Maybe we should make this configurable? any suggestions?
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. Network congestion is certainly a possibility, I think that for now this strike a good balance between simple code and avoiding hanging all transfers if we have one slow target host. We can revisit this in the future if it turns out we need more control in production environments. Sound good? |
||
| blockId, | ||
| existingReplicas.toSet, | ||
| maxReplicas, | ||
| forceFetch = false, | ||
| maxReplicationFailures = Some(maxReplicationFailures)) | ||
| if (replicatedSuccessfully) { | ||
| logInfo(s"Block $blockId offloaded successfully, Removing block now") | ||
| removeBlock(blockId) | ||
| logInfo(s"Block $blockId removed") | ||
| } else { | ||
| logWarning(s"Failed to offload block $blockId") | ||
| } | ||
| replicatedSuccessfully | ||
| } | ||
| if (blocksFailedReplication.nonEmpty) { | ||
| logWarning(s"Blocks failed replication in cache decommissioning " + | ||
| s"process: ${blocksFailedReplication.mkString(",")}") | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Remove all blocks belonging to the given broadcast. | ||
| */ | ||
|
|
@@ -1829,7 +1895,45 @@ private[spark] class BlockManager( | |
| data.dispose() | ||
| } | ||
|
|
||
| class BlockManagerDecommissionManager { | ||
|
prakharjain09 marked this conversation as resolved.
Outdated
|
||
| @volatile private var stopped = false | ||
| private val cacheReplicationThread = new Thread { | ||
|
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 comment about the thread up above, I'm not sure why we need to be in a separate thread (but maybe I've missed something).
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. Tried to explain why we have created a thread here - #27864 (comment). Basically this BlockManager will keep on retrying offloading of cache blocks every 30 seconds.
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. Makes sense. I guess we can just (later on) send a message back once all the blocks are done migrating.
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. When all the blocks are done migrating - Driver will automatically know. BlockManagerMaster is continuously receiving BlockUpdates from this BlockManager (using the UpdateBlockInfo message). So when all blocks are offloaded - Dynamic allocation will atomatically know that this executor doesn't have cache data and If other criteria are met (like executorIdleTimeout etc), then the executor will automatically be removed by ExecutorMonitor class of DynamicAllocation. |
||
| override def run(): Unit = { | ||
| while (blockManagerDecommissioning && !stopped) { | ||
| try { | ||
| logDebug(s"Attempting to replicate all cached RDD blocks") | ||
|
prakharjain09 marked this conversation as resolved.
Outdated
|
||
| offloadRddCacheBlocks() | ||
| logInfo(s"Attempt to replicate all cached blocks done") | ||
|
prakharjain09 marked this conversation as resolved.
Outdated
|
||
| Thread.sleep(30000) | ||
| } catch { | ||
| case _: InterruptedException => | ||
| // no-op | ||
|
prakharjain09 marked this conversation as resolved.
Outdated
|
||
| case NonFatal(e) => | ||
| logError("Error occurred while trying to " + | ||
| "replicate cached RDD blocks for block manager decommissioning", e) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| cacheReplicationThread.setDaemon(true) | ||
| cacheReplicationThread.setName("cache-replication-thread") | ||
|
|
||
| def start(): Unit = { | ||
|
prakharjain09 marked this conversation as resolved.
|
||
| cacheReplicationThread.start() | ||
| } | ||
|
|
||
| def stop(): Unit = { | ||
| if (!stopped) { | ||
| stopped = true | ||
| logInfo("Stopping cache replication thread") | ||
| cacheReplicationThread.interrupt() | ||
| cacheReplicationThread.join() | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def stop(): Unit = { | ||
| decommissionManager.foreach(_.stop()) | ||
| blockTransferService.close() | ||
| if (blockStoreClient ne blockTransferService) { | ||
| // Closing should be idempotent, but maybe not for the NioBlockTransferService. | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.