-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-20732][CORE] Decommission cache blocks to other executors when an executor is decommissioned #28370
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 #28370
Changes from all commits
ab0e38c
6a47615
622e1ba
d792092
b9906c2
076dd67
d12dbff
4c67660
f6b4f7c
9c6bdb6
bb324f9
12e865c
5847c1c
a2a81f6
3a14320
6ab11e3
c645582
37ad189
b365921
75d6daa
c343056
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 | ||
|
@@ -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( | ||
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. @prakharjain09 / @holdenk, Thank you for this improvement. I had a question please: (I am still new to this code paths and I am not totally sure of what I am talking about. So if there is something I am missing please help me fill the gaps :-). ) I notice that I understand it is a bit late to do the replication when the executor is indeed lost: Since decommissioning as implemented in #26440 does not really trigger eager executor loss. We instead merely stop scheduling on the decom'd executor and let it be shot down out of band. Which means that the replication triggered in SPARK-15355 would be too late. I like the approach taken in this PR to eagerly tell the executor (block-manager) to start replication when the decom is first initiated, to give it more time to be useful. But I wonder if you implemented this somewhat differently by leveraging the existing eager replication loop ?. Thanks ! 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. So the existing block replication is for the case where blocks we stored on two machines and due to executor loss are now down to one machine so they are replicated. It's not useless but it doesn't solve the same core problem. |
||
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 => | ||
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. use 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. Using 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 see. |
||
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 | ||
prakharjain09 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
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) | ||
|
@@ -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, | ||
|
@@ -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 | ||
|
@@ -1641,6 +1653,10 @@ private[spark] class BlockManager( | |
peersForReplication = peersForReplication.tail | ||
peersReplicatedTo += peer | ||
} catch { | ||
// Rethrow interrupt exception | ||
case e: InterruptedException => | ||
throw e | ||
// Everything else we may retry | ||
case NonFatal(e) => | ||
logWarning(s"Failed to replicate $blockId to $peer, failure #$numFailures", e) | ||
peersFailedToReplicateTo += peer | ||
|
@@ -1665,9 +1681,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 | ||
} | ||
|
||
/** | ||
|
@@ -1761,6 +1779,60 @@ 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 = { | ||
prakharjain09 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
val replicateBlocksInfo = master.getReplicateInfoForRDDBlocks(blockManagerId) | ||
|
||
if (replicateBlocksInfo.nonEmpty) { | ||
logInfo(s"Need to replicate ${replicateBlocksInfo.size} blocks " + | ||
"for block manager decommissioning") | ||
} else { | ||
logWarning(s"Asked to decommission RDD cache blocks, but no blocks to migrate") | ||
return | ||
} | ||
|
||
// 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 = replicateBlocksInfo.map { | ||
case ReplicateBlock(blockId, existingReplicas, maxReplicas) => | ||
val replicatedSuccessfully = replicateBlock( | ||
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. | ||
*/ | ||
|
@@ -1829,7 +1901,58 @@ 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) { | ||
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. Do we really need a wrapped manager class? It seems overkill to me. 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. For the first part I'm ambivalent, but given that we also want to migrate shuffle blocks after I think having a manager is ok. 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. We should implement it step by step and could always do refactor later. Or, we should at least add a todo ticket to explain why we need this and what we plan to do next. Otherwise, I am really -1 on this kind of change. 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. So there are two conversations I want to have about this with you @Ngone51 now to make sure I'm understanding what you're trying to express. There already is a second follow up PR that extends the I want to understand your -1 here because that has some pretty strong meanings in the context of a code change. A -1 is generally viewed as expressing a veto, which I don't believe you have in the project (of course I was out for a month in the hospital last year so if you do please let point me to thread). Even if you don't have a veto in the project is it your intention to say that if you did have a veto you would block this code change? A veto is generally a very strong expression, and I'm worried I'm not understanding your reasoning since this seems like a relatively minor issue. 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. Also, I understand text-only communication can have more misunderstandings, if you want to find a time this week when we're both free to jump on a call to clarify this (and we can write back our understanding here so it's recorded for people to understand what we talked about), I'd be more than happy to. 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. For a new reviewer (e.g. me) on a big topic, it's not always possible to know every detail(even worse, when there's no design doc). So it's the author's responsibility to give more context. For example, leaving todo JIRA tickets in the code comment or reply to give more information. But without sufficient context here, I really think "this change", wrapping a manager around a thread, doesn't make sense to me. As for "-1", it really represents my personal opinion. I should say "I don't like this change" if "-1" means a lot for the community. 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. As a reviewer it’s expected that you would read the issue before asking for a follow up issue in a blocking manner. 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. Of course, I did. But I still don't get it and I think it' not always possible that a reviewer could know the sub-issue is mean to be a follow up for some specific codes without design document/code comments around here. 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. So if you look at the parent issue you can see there is another sub issue that says migrate shuffle blocks. It’s ok to ask for a follow up even if there is one (we all miss things in reading), but attempt to vote a -1 has a higher bar than just asking for something. |
||
@volatile private var stopped = false | ||
private val sleepInterval = conf.get( | ||
config.STORAGE_DECOMMISSION_REPLICATION_REATTEMPT_INTERVAL) | ||
|
||
private val blockReplicationThread = new Thread { | ||
override def run(): Unit = { | ||
var failures = 0 | ||
while (blockManagerDecommissioning | ||
&& !stopped | ||
&& !Thread.interrupted() | ||
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. Do we really need this?
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. Unless the interrupt exception is caught inside of the block transfer 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.
If there's an If you want the decommission thread stop, then, Or if you want the decommission thread to keep working, then, 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. If an interrupt exception is caught the thread would still be marked as interrupted 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 believe in normal cases like
And even if the status is not cleared in other cases, the following 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. So that block of text indicates that the interrupted status is cleared when the exception is thrown. Which means it would be possible for the thread to be in an interrupted status without having the exception thrown. Worst case scenario: this check is not needed but also causes no significant harm |
||
&& failures < 20) { | ||
try { | ||
logDebug("Attempting to replicate all cached RDD blocks") | ||
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. Could we add attempt number to the log? 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. |
||
decommissionRddCacheBlocks() | ||
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. Don't you need to set Or you mean we need to do multiple time 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. We don't set |
||
logInfo("Attempt to replicate all cached blocks done") | ||
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. attempt number? 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’d say fine to do in a follow up but if we want to add the attempt number here go for it (but I won’t hold off on merging for that). |
||
Thread.sleep(sleepInterval) | ||
} catch { | ||
case _: InterruptedException => | ||
logInfo("Interrupted during migration, will not refresh migrations.") | ||
stopped = true | ||
case NonFatal(e) => | ||
failures += 1 | ||
logError("Error occurred while trying to replicate cached RDD blocks" + | ||
s" for block manager decommissioning (failure count: $failures)", e) | ||
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. attempt number? |
||
} | ||
} | ||
} | ||
} | ||
blockReplicationThread.setDaemon(true) | ||
blockReplicationThread.setName("block-replication-thread") | ||
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. Use 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. Looking at our code we seem to be roughly split on 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. We always use 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.
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. Ah..that's good point but just wondering how many of them are chosen after realizing BTW, you'd better grep "new Thread(" to exclude ThreadLocal declaration. 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. Even that returns 36 in core. |
||
|
||
def start(): Unit = { | ||
logInfo("Starting block replication thread") | ||
blockReplicationThread.start() | ||
} | ||
|
||
def stop(): Unit = { | ||
if (!stopped) { | ||
stopped = true | ||
logInfo("Stopping block replication thread") | ||
blockReplicationThread.interrupt() | ||
} | ||
} | ||
} | ||
|
||
def stop(): Unit = { | ||
decommissionManager.foreach(_.stop()) | ||
blockTransferService.close() | ||
if (blockStoreClient ne blockTransferService) { | ||
// Closing should be idempotent, but maybe not for the NioBlockTransferService. | ||
|
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 think we can just use
spark.storage.maxReplicationFailures
directly. Less configurations contribute to better UX.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.
So I'm not sure that's a great idea. Looking at
maxReplicationFailures
the default is set to one, which certainly makes sense in the situation where we don't expect the host to be exiting. But this situation is different, we know the current block is going to disappear soon so it makes sense to more aggressively try and copy the block.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 see, thanks for your explanation.