Skip to content

[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

Closed
28 changes: 28 additions & 0 deletions core/src/main/scala/org/apache/spark/internal/config/package.scala
Original file line number Diff line number Diff line change
Expand Up @@ -413,6 +413,34 @@ package object config {
.intConf
.createWithDefault(1)

private[spark] val STORAGE_DECOMMISSION_ENABLED =
ConfigBuilder("spark.storage.decommission.enabled")
.doc("Whether to decommission the block manager when decommissioning executor")
Copy link
Member

Choose a reason for hiding this comment

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

Could you explain about the meaning of decommission here?

Let say, we have Spark Thrift Server. In that case, dynamic allocation wants to decommission all workers. Then, what this PR can provide for zero-worker situation?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@dongjoon-hyun Thanks for the detailed review.

I couldn't understand your question with respect to dynamic allocation completely (as per my understanding dynamic allocation doesn't initiates decommissioning). This change works together with SPARK-20628 PR. After SPARK-20628, Spark scheduler starts receiving decommissionExecutor message (because of AWS spot loss, Azure low-priority VMs etc). In such case, we should offload cache data from that executor to other active executors so that we don't eventually loose those cache blocks. This is done in a best effort way i.e. move cache blocks if space is available on active executors. If space is not available on other active executors, then keep the cache blocks.

Next step after this change is to initiate decommissioning from DynamicAllocation. Let say minExecutor=2, maxExecutor=50. A spark application is running at max 50 executors and all of them have little bit of cache data (say 2GB out of total available 5GB capacity). So DynamicAllocation is not able to downscale any of the executors (although they are idle). In future, DynamicAllocation can leverage changes in this PR to do defragmentation of cache data to fewer set of executors so that some of the executors can be freed up.

Hope this answers your question.

Copy link
Contributor

Choose a reason for hiding this comment

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

Since this is focused on executor to executor migration supporting zero executors isn't going to be a viable option here. However since were using the same methods for putting blocks, assuming that we add support for storing these blocks in external storage in the future, this approach should be able to be generalized to use that same mechanism if configured.

.version("3.1.0")
.booleanConf
.createWithDefault(false)

private[spark] val STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK =
ConfigBuilder("spark.storage.decommission.maxReplicationFailuresPerBlock")
.internal()
.doc("Maximum number of failures which can be handled for the replication of " +
"one RDD block when block manager is decommissioning and trying to move its " +
"existing blocks.")
.version("3.1.0")
.intConf
.createWithDefault(3)

private[spark] val STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL =
ConfigBuilder("spark.storage.decommission.replicationReattemptInterval")
.internal()
.doc("The interval of time between consecutive cache block replication reattempts " +
"happening on each decommissioning executor (due to storage decommissioning).")
.version("3.1.0")
.timeConf(TimeUnit.MILLISECONDS)
.checkValue(_ > 0, "Time interval between two consecutive attempts of " +
"cache block replication should be positive.")
.createWithDefaultString("30s")

private[spark] val STORAGE_REPLICATION_TOPOLOGY_FILE =
ConfigBuilder("spark.storage.replication.topologyFile")
.version("2.1.0")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -438,6 +438,19 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
logError(s"Unexpected error during decommissioning ${e.toString}", e)
}
logInfo(s"Finished decommissioning executor $executorId.")

if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
try {
logInfo("Starting decommissioning block manager corresponding to " +
s"executor $executorId.")
scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
} catch {
case e: Exception =>
logError("Unexpected error during block manager " +
s"decommissioning for executor $executorId: ${e.toString}", e)
}
logInfo(s"Acknowledged decommissioning block manager corresponding to $executorId.")
}
} else {
logInfo(s"Skipping decommissioning of executor $executorId.")
}
Expand Down Expand Up @@ -574,7 +587,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
*/
private[spark] def decommissionExecutor(executorId: String): Unit = {
if (driverEndpoint != null) {
logInfo("Propegating executor decommission to driver.")
logInfo("Propagating executor decommission to driver.")
driverEndpoint.send(DecommissionExecutor(executorId))
}
}
Expand Down Expand Up @@ -658,7 +671,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
/**
* Update the cluster manager on our scheduling needs. Three bits of information are included
* to help it make decisions.
* @param resourceProfileToNumExecutors The total number of executors we'd like to have per
* @param resourceProfileIdToNumExecutors The total number of executors we'd like to have per
* ResourceProfile. The cluster manager shouldn't kill any
* running executor to reach this number, but, if all
* existing executors were to die, this is the number
Expand Down
129 changes: 120 additions & 9 deletions core/src/main/scala/org/apache/spark/storage/BlockManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -1551,30 +1555,36 @@ private[spark] class BlockManager(
}

/**
* Called for pro-active replenishment of blocks lost due to executor failures
* Replicates a block to peer block managers based on existingReplicas and maxReplicas
*
* @param blockId blockId being replicate
* @param existingReplicas existing block managers that have a replica
* @param maxReplicas maximum replicas needed
* @param maxReplicationFailures number of replication failures to tolerate before
* giving up.
* @return whether block was successfully replicated or not
*/
def replicateBlock(
blockId: BlockId,
existingReplicas: Set[BlockManagerId],
maxReplicas: Int): Unit = {
maxReplicas: Int,
maxReplicationFailures: Option[Int] = None): Boolean = {
logInfo(s"Using $blockManagerId to pro-actively replicate $blockId")
blockInfoManager.lockForReading(blockId).foreach { info =>
blockInfoManager.lockForReading(blockId).forall { info =>
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
// we know we are called as a result of an executor removal or because the current executor
// is getting decommissioned. so we refresh peer cache before trying replication, we won't
// try to replicate to a missing executor/another decommissioning executor
getPeers(forceFetch = true)
try {
replicate(blockId, data, storageLevel, info.classTag, existingReplicas)
replicate(
blockId, data, storageLevel, info.classTag, existingReplicas, maxReplicationFailures)
} finally {
logDebug(s"Releasing lock for $blockId")
releaseLockAndDispose(blockId, data)
Expand All @@ -1591,9 +1601,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,
Expand All @@ -1617,7 +1629,7 @@ private[spark] class BlockManager(
blockId,
numPeersToReplicateTo)

while(numFailures <= maxReplicationFailures &&
while(numFailures <= maxReplicationFailureCount &&
!peersForReplication.isEmpty &&
peersReplicatedTo.size < numPeersToReplicateTo) {
val peer = peersForReplication.head
Expand Down Expand Up @@ -1665,9 +1677,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(", ")}")
return true
}

/**
Expand Down Expand Up @@ -1761,6 +1775,58 @@ private[spark] class BlockManager(
blocksToRemove.size
}

def decommissionBlockManager(): Unit = {
if (!blockManagerDecommissioning) {
logInfo("Starting block manager decommissioning process")
blockManagerDecommissioning = true
decommissionManager = Some(new BlockManagerDecommissionManager(conf))
decommissionManager.foreach(_.start())
} else {
logDebug("Block manager already in decommissioning state")
}
}

/**
* Tries to offload all cached RDD blocks from this BlockManager to peer BlockManagers
* Visible for testing
*/
def decommissionRddCacheBlocks(): Unit = {
val replicateBlocksInfo = master.getReplicateInfoForRDDBlocks(blockManagerId)

if (replicateBlocksInfo.nonEmpty) {
logInfo(s"Need to replicate ${replicateBlocksInfo.size} blocks " +
"for block manager decommissioning")
}

// Maximum number of storage replication failure which replicateBlock can handle
val maxReplicationFailures = conf.get(
config.STORAGE_DECOMMISSION_MAX_REPLICATION_FAILURE_PER_BLOCK)

// TODO: We can sort these blocks based on some policy (LRU/blockSize etc)
// so that we end up prioritize them over each other
val blocksFailedReplication = ThreadUtils.parmap(
replicateBlocksInfo, "decommissionRddCacheBlocks", 4) {
case ReplicateBlock(blockId, existingReplicas, maxReplicas) =>
val replicatedSuccessfully = replicateBlock(
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 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.

Copy link
Contributor Author

@prakharjain09 prakharjain09 Apr 23, 2020

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

The 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,
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")
}
(blockId, replicatedSuccessfully)
}.filterNot(_._2).map(_._1)
if (blocksFailedReplication.nonEmpty) {
logWarning("Blocks failed replication in cache decommissioning " +
s"process: ${blocksFailedReplication.mkString(",")}")
}
}

/**
* Remove all blocks belonging to the given broadcast.
*/
Expand Down Expand Up @@ -1829,7 +1895,52 @@ private[spark] class BlockManager(
data.dispose()
}

/**
* Class to handle block manager decommissioning retries
* It creates a Thread to retry offloading all RDD cache blocks
*/
private class BlockManagerDecommissionManager(conf: SparkConf) {
@volatile private var stopped = false
private val blockReplicationThread = new Thread {
override def run(): Unit = {
while (blockManagerDecommissioning && !stopped) {
try {
logDebug("Attempting to replicate all cached RDD blocks")
decommissionRddCacheBlocks()
logInfo("Attempt to replicate all cached blocks done")
val sleepInterval = conf.get(
config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL)
Thread.sleep(sleepInterval)
} catch {
case _: InterruptedException =>
// no-op
case NonFatal(e) =>
logError("Error occurred while trying to " +
"replicate cached RDD blocks for block manager decommissioning", e)
}
}
}
}
blockReplicationThread.setDaemon(true)
blockReplicationThread.setName("block-replication-thread")

def start(): Unit = {
logInfo("Starting block replication thread")
blockReplicationThread.start()
}

def stop(): Unit = {
if (!stopped) {
stopped = true
logInfo("Stopping block replication thread")
blockReplicationThread.interrupt()
blockReplicationThread.join()
}
}
}

def stop(): Unit = {
decommissionManager.foreach(_.stop())
blockTransferService.close()
if (blockStoreClient ne blockTransferService) {
// Closing should be idempotent, but maybe not for the NioBlockTransferService.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,16 @@ class BlockManagerMaster(
logInfo("Removed " + execId + " successfully in removeExecutor")
}

/** Decommission block managers corresponding to given set of executors */
def decommissionBlockManagers(executorIds: Seq[String]): Unit = {
driverEndpoint.ask[Unit](DecommissionBlockManagers(executorIds))
}

/** Get Replication Info for all the RDD blocks stored in given blockManagerId */
def getReplicateInfoForRDDBlocks(blockManagerId: BlockManagerId): Seq[ReplicateBlock] = {
driverEndpoint.askSync[Seq[ReplicateBlock]](GetReplicateInfoForRDDBlocks(blockManagerId))
}

/** Request removal of a dead executor from the driver endpoint.
* This is only called on the driver side. Non-blocking
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,9 @@ class BlockManagerMasterEndpoint(
// Mapping from executor ID to block manager ID.
private val blockManagerIdByExecutor = new mutable.HashMap[String, BlockManagerId]

// Set of block managers which are decommissioning
private val decommissioningBlockManagerSet = new mutable.HashSet[BlockManagerId]

// Mapping from block id to the set of block managers that have the block.
private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]]

Expand Down Expand Up @@ -153,6 +156,13 @@ class BlockManagerMasterEndpoint(
removeExecutor(execId)
context.reply(true)

case DecommissionBlockManagers(executorIds) =>
decommissionBlockManagers(executorIds.flatMap(blockManagerIdByExecutor.get))
context.reply(true)

case GetReplicateInfoForRDDBlocks(blockManagerId) =>
context.reply(getReplicateInfoForRDDBlocks(blockManagerId))

case StopBlockManagerMaster =>
context.reply(true)
stop()
Expand Down Expand Up @@ -257,6 +267,7 @@ class BlockManagerMasterEndpoint(

// Remove the block manager from blockManagerIdByExecutor.
blockManagerIdByExecutor -= blockManagerId.executorId
decommissioningBlockManagerSet.remove(blockManagerId)

// Remove it from blockManagerInfo and remove all the blocks.
blockManagerInfo.remove(blockManagerId)
Expand Down Expand Up @@ -299,6 +310,39 @@ class BlockManagerMasterEndpoint(
blockManagerIdByExecutor.get(execId).foreach(removeBlockManager)
}

/**
* Decommission the given Seq of blockmanagers
* - Adds these block managers to decommissioningBlockManagerSet Set
* - Sends the DecommissionBlockManager message to each of the [[BlockManagerSlaveEndpoint]]
*/
def decommissionBlockManagers(blockManagerIds: Seq[BlockManagerId]): Future[Seq[Unit]] = {
val newBlockManagersToDecommission = blockManagerIds.toSet.diff(decommissioningBlockManagerSet)
val futures = newBlockManagersToDecommission.map { blockManagerId =>
decommissioningBlockManagerSet.add(blockManagerId)
val info = blockManagerInfo(blockManagerId)
info.slaveEndpoint.ask[Unit](DecommissionBlockManager)
}
Future.sequence{ futures.toSeq }
}

/**
* Returns a Seq of ReplicateBlock for each RDD block stored by given blockManagerId
* @param blockManagerId - block manager id for which ReplicateBlock info is needed
* @return Seq of ReplicateBlock
*/
private def getReplicateInfoForRDDBlocks(blockManagerId: BlockManagerId): Seq[ReplicateBlock] = {
val info = blockManagerInfo(blockManagerId)

val rddBlocks = info.blocks.keySet().asScala.filter(_.isRDD)
rddBlocks.map { blockId =>
val currentBlockLocations = blockLocations.get(blockId)
val maxReplicas = currentBlockLocations.size + 1
val remainingLocations = currentBlockLocations.toSeq.filter(bm => bm != blockManagerId)
val replicateMsg = ReplicateBlock(blockId, remainingLocations, maxReplicas)
replicateMsg
}.toSeq
}

// Remove a block from the slaves that have it. This can only be used to remove
// blocks that the master knows about.
private def removeBlockFromWorkers(blockId: BlockId): Unit = {
Expand Down Expand Up @@ -536,7 +580,11 @@ class BlockManagerMasterEndpoint(
private def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = {
val blockManagerIds = blockManagerInfo.keySet
if (blockManagerIds.contains(blockManagerId)) {
blockManagerIds.filterNot { _.isDriver }.filterNot { _ == blockManagerId }.toSeq
blockManagerIds
.filterNot { _.isDriver }
.filterNot { _ == blockManagerId }
.diff(decommissioningBlockManagerSet)
.toSeq
} else {
Seq.empty
}
Expand Down
Loading