-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-53446][CORE] Optimize BlockManager remove operations with cached block mappings #52210
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
base: master
Are you sure you want to change the base?
Changes from all commits
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 | ||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -288,6 +288,65 @@ private[spark] class BlockManager( | |||||||||||||||||
| securityManager.getIOEncryptionKey()) | ||||||||||||||||||
| private val maxRemoteBlockToMem = conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) | ||||||||||||||||||
|
|
||||||||||||||||||
| // SPARK-53446 Performance optimization: Cache mappings to avoid O(n) scans in remove operations | ||||||||||||||||||
| private[this] val rddToBlockIds = | ||||||||||||||||||
| new ConcurrentHashMap[Int, ConcurrentHashMap.KeySetView[BlockId, java.lang.Boolean]]() | ||||||||||||||||||
| private[this] val broadcastToBlockIds = | ||||||||||||||||||
| new ConcurrentHashMap[Long, ConcurrentHashMap.KeySetView[BlockId, java.lang.Boolean]]() | ||||||||||||||||||
| private[this] val sessionToBlockIds = | ||||||||||||||||||
| new ConcurrentHashMap[String, ConcurrentHashMap.KeySetView[BlockId, java.lang.Boolean]]() | ||||||||||||||||||
|
|
||||||||||||||||||
| /** | ||||||||||||||||||
| * Add a block ID to the appropriate cache mapping based on its type. | ||||||||||||||||||
| */ | ||||||||||||||||||
| private def addToCache(blockId: BlockId): Unit = { | ||||||||||||||||||
| blockId match { | ||||||||||||||||||
| case rddBlockId: RDDBlockId => | ||||||||||||||||||
| rddToBlockIds | ||||||||||||||||||
| .computeIfAbsent(rddBlockId.rddId, _ => ConcurrentHashMap.newKeySet()) | ||||||||||||||||||
| .add(blockId) | ||||||||||||||||||
| case broadcastBlockId: BroadcastBlockId => | ||||||||||||||||||
| broadcastToBlockIds | ||||||||||||||||||
| .computeIfAbsent(broadcastBlockId.broadcastId, _ => ConcurrentHashMap.newKeySet()) | ||||||||||||||||||
| .add(blockId) | ||||||||||||||||||
| case cacheId: CacheId => | ||||||||||||||||||
| sessionToBlockIds | ||||||||||||||||||
| .computeIfAbsent(cacheId.sessionUUID, _ => ConcurrentHashMap.newKeySet()) | ||||||||||||||||||
| .add(blockId) | ||||||||||||||||||
| case _ => // Do nothing for other block types | ||||||||||||||||||
| } | ||||||||||||||||||
| } | ||||||||||||||||||
|
|
||||||||||||||||||
| /** | ||||||||||||||||||
| * Remove a block ID from the appropriate cache mapping based on its type. | ||||||||||||||||||
| */ | ||||||||||||||||||
| private def removeFromCache(blockId: BlockId): Unit = { | ||||||||||||||||||
| blockId match { | ||||||||||||||||||
| case rddBlockId: RDDBlockId => | ||||||||||||||||||
| Option(rddToBlockIds.get(rddBlockId.rddId)).foreach { blockSet => | ||||||||||||||||||
| blockSet.remove(blockId) | ||||||||||||||||||
| if (blockSet.isEmpty) { | ||||||||||||||||||
| rddToBlockIds.remove(rddBlockId.rddId) | ||||||||||||||||||
|
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 think there is a race condition 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. Agree with @cloud-fan, use |
||||||||||||||||||
| } | ||||||||||||||||||
| } | ||||||||||||||||||
| case broadcastBlockId: BroadcastBlockId => | ||||||||||||||||||
| Option(broadcastToBlockIds.get(broadcastBlockId.broadcastId)).foreach { blockSet => | ||||||||||||||||||
| blockSet.remove(blockId) | ||||||||||||||||||
| if (blockSet.isEmpty) { | ||||||||||||||||||
| broadcastToBlockIds.remove(broadcastBlockId.broadcastId) | ||||||||||||||||||
| } | ||||||||||||||||||
| } | ||||||||||||||||||
| case cacheId: CacheId => | ||||||||||||||||||
| Option(sessionToBlockIds.get(cacheId.sessionUUID)).foreach { blockSet => | ||||||||||||||||||
| blockSet.remove(blockId) | ||||||||||||||||||
| if (blockSet.isEmpty) { | ||||||||||||||||||
| sessionToBlockIds.remove(cacheId.sessionUUID) | ||||||||||||||||||
| } | ||||||||||||||||||
| } | ||||||||||||||||||
| case _ => // Do nothing for other block types | ||||||||||||||||||
| } | ||||||||||||||||||
| } | ||||||||||||||||||
|
|
||||||||||||||||||
| var hostLocalDirManager: Option[HostLocalDirManager] = None | ||||||||||||||||||
|
|
||||||||||||||||||
| @inline final private def isDecommissioning() = { | ||||||||||||||||||
|
|
@@ -1560,6 +1619,7 @@ private[spark] class BlockManager( | |||||||||||||||||
| exceptionWasThrown = false | ||||||||||||||||||
| if (res.isEmpty) { | ||||||||||||||||||
| // the block was successfully stored | ||||||||||||||||||
| addToCache(blockId) | ||||||||||||||||||
|
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. is 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 adding a new block, yes; it goes through |
||||||||||||||||||
| if (keepReadLock) { | ||||||||||||||||||
| blockInfoManager.downgradeLock(blockId) | ||||||||||||||||||
| } else { | ||||||||||||||||||
|
|
@@ -2028,9 +2088,13 @@ private[spark] class BlockManager( | |||||||||||||||||
| * @return The number of blocks removed. | ||||||||||||||||||
| */ | ||||||||||||||||||
| def removeRdd(rddId: Int): Int = { | ||||||||||||||||||
| // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks. | ||||||||||||||||||
| logInfo(log"Removing RDD ${MDC(RDD_ID, rddId)}") | ||||||||||||||||||
| val blocksToRemove = blockInfoManager.entries.flatMap(_._1.asRDDId).filter(_.rddId == rddId) | ||||||||||||||||||
| val blocksToRemove = Option(rddToBlockIds.get(rddId)) match { | ||||||||||||||||||
| case Some(blockSet) => | ||||||||||||||||||
| blockSet.asScala.toSeq | ||||||||||||||||||
| case None => | ||||||||||||||||||
| Seq.empty | ||||||||||||||||||
| } | ||||||||||||||||||
|
Comment on lines
+2092
to
+2097
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. Remove it proactively from the map. This also makes the
Suggested change
|
||||||||||||||||||
| blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false) } | ||||||||||||||||||
| blocksToRemove.size | ||||||||||||||||||
| } | ||||||||||||||||||
|
|
@@ -2064,8 +2128,11 @@ private[spark] class BlockManager( | |||||||||||||||||
| */ | ||||||||||||||||||
| def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = { | ||||||||||||||||||
| logDebug(s"Removing broadcast $broadcastId") | ||||||||||||||||||
| val blocksToRemove = blockInfoManager.entries.map(_._1).collect { | ||||||||||||||||||
| case bid @ BroadcastBlockId(`broadcastId`, _) => bid | ||||||||||||||||||
| val blocksToRemove = Option(broadcastToBlockIds.get(broadcastId)) match { | ||||||||||||||||||
| case Some(blockSet) => | ||||||||||||||||||
| blockSet.asScala.toSeq | ||||||||||||||||||
| case None => | ||||||||||||||||||
| Seq.empty | ||||||||||||||||||
| } | ||||||||||||||||||
| blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster) } | ||||||||||||||||||
| blocksToRemove.size | ||||||||||||||||||
|
|
@@ -2078,8 +2145,11 @@ private[spark] class BlockManager( | |||||||||||||||||
| */ | ||||||||||||||||||
| def removeCache(sessionUUID: String): Int = { | ||||||||||||||||||
| logDebug(s"Removing cache of spark session with UUID: $sessionUUID") | ||||||||||||||||||
| val blocksToRemove = blockInfoManager.entries.map(_._1).collect { | ||||||||||||||||||
| case cid: CacheId if cid.sessionUUID == sessionUUID => cid | ||||||||||||||||||
| val blocksToRemove = Option(sessionToBlockIds.get(sessionUUID)) match { | ||||||||||||||||||
| case Some(blockSet) => | ||||||||||||||||||
| blockSet.asScala.toSeq | ||||||||||||||||||
| case None => | ||||||||||||||||||
| Seq.empty | ||||||||||||||||||
| } | ||||||||||||||||||
| blocksToRemove.foreach { blockId => removeBlock(blockId) } | ||||||||||||||||||
| blocksToRemove.size | ||||||||||||||||||
|
|
@@ -2122,6 +2192,7 @@ private[spark] class BlockManager( | |||||||||||||||||
| } | ||||||||||||||||||
|
|
||||||||||||||||||
| blockInfoManager.removeBlock(blockId) | ||||||||||||||||||
| removeFromCache(blockId) | ||||||||||||||||||
| hasRemoveBlock = true | ||||||||||||||||||
|
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 this in the finally block as well. |
||||||||||||||||||
| if (tellMaster) { | ||||||||||||||||||
| // Only update storage level from the captured block status before deleting, so that | ||||||||||||||||||
|
|
||||||||||||||||||
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.
nit: Move these helper methods towards the bottom.