Skip to content

Commit

Permalink
[SPARK-2316] Avoid O(blocks) operations in listeners
Browse files Browse the repository at this point in the history
The existing code in `StorageUtils` is not the most efficient. Every time we want to update an `RDDInfo` we end up iterating through all blocks on all block managers just to discard most of them. The symptoms manifest themselves in the bountiful UI bugs observed in the wild. Many of these bugs are caused by the slow consumption of events in `LiveListenerBus`, which frequently leads to the event queue overflowing and `SparkListenerEvent`s being dropped on the floor. The changes made in this PR avoid this by first filtering out only the blocks relevant to us before computing storage information from them.

It's worth a mention that this corner of the Spark code is also not very well-tested at all. The bulk of the changes in this PR (more than 60%) is actually test cases for the various logic in `StorageUtils.scala` as well as `StorageTab.scala`. These will eventually be extended to cover the various listeners that constitute the `SparkUI`.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1679 from andrewor14/fix-drop-events and squashes the following commits:

f80c1fa [Andrew Or] Rewrite fold and reduceOption as sum
e132d69 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events
14fa1c3 [Andrew Or] Simplify some code + update a few comments
a91be46 [Andrew Or] Make ExecutorsPage blazingly fast
bf6f09b [Andrew Or] Minor changes
8981de1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events
af19bc0 [Andrew Or] *UsedByRDD -> *UsedByRdd (minor)
6970bc8 [Andrew Or] Add extensive tests for StorageListener and the new code in StorageUtils
e080b9e [Andrew Or] Reduce run time of StorageUtils.updateRddInfo to near constant
2c3ef6a [Andrew Or] Actually filter out only the relevant RDDs
6fef86a [Andrew Or] Add extensive tests for new code in StorageStatus
b66b6b0 [Andrew Or] Use more efficient underlying data structures for blocks
6a7b7c0 [Andrew Or] Avoid chained operations on TraversableLike
a9ec384 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events
b12fcd7 [Andrew Or] Fix tests + simplify sc.getRDDStorageInfo
da8e322 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events
8e91921 [Andrew Or] Iterate through a filtered set of blocks when updating RDDInfo
7b2c4aa [Andrew Or] Rewrite blockLocationsFromStorageStatus + clean up method signatures
41fa50d [Andrew Or] Add a legacy constructor for StorageStatus
53af15d [Andrew Or] Refactor StorageStatus + add a bunch of tests
  • Loading branch information
andrewor14 authored and pwendell committed Aug 2, 2014
1 parent dab3796 commit d934801
Show file tree
Hide file tree
Showing 13 changed files with 843 additions and 176 deletions.
6 changes: 4 additions & 2 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ import org.apache.spark.scheduler._
import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend}
import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
import org.apache.spark.scheduler.local.LocalBackend
import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils}
import org.apache.spark.storage._
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils}

Expand Down Expand Up @@ -843,7 +843,9 @@ class SparkContext(config: SparkConf) extends Logging {
*/
@DeveloperApi
def getRDDStorageInfo: Array[RDDInfo] = {
StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this)
val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray
StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus)
rddInfos.filter(_.isCached)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -267,9 +267,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
}

private def storageStatus: Array[StorageStatus] = {
blockManagerInfo.map { case(blockManagerId, info) =>
val blockMap = mutable.Map[BlockId, BlockStatus](info.blocks.toSeq: _*)
new StorageStatus(blockManagerId, info.maxMem, blockMap)
blockManagerInfo.map { case (blockManagerId, info) =>
new StorageStatus(blockManagerId, info.maxMem, info.blocks)
}.toArray
}

Expand Down Expand Up @@ -424,7 +423,14 @@ case class BlockStatus(
storageLevel: StorageLevel,
memSize: Long,
diskSize: Long,
tachyonSize: Long)
tachyonSize: Long) {
def isCached: Boolean = memSize + diskSize + tachyonSize > 0
}

@DeveloperApi
object BlockStatus {
def empty: BlockStatus = BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)
}

private[spark] class BlockManagerInfo(
val blockManagerId: BlockManagerId,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,36 +30,32 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar
metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] {
override def getValue: Long = {
val storageStatusList = blockManager.master.getStorageStatus
val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _)
val maxMem = storageStatusList.map(_.maxMem).sum
maxMem / 1024 / 1024
}
})

metricRegistry.register(MetricRegistry.name("memory", "remainingMem_MB"), new Gauge[Long] {
override def getValue: Long = {
val storageStatusList = blockManager.master.getStorageStatus
val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _)
val remainingMem = storageStatusList.map(_.memRemaining).sum
remainingMem / 1024 / 1024
}
})

metricRegistry.register(MetricRegistry.name("memory", "memUsed_MB"), new Gauge[Long] {
override def getValue: Long = {
val storageStatusList = blockManager.master.getStorageStatus
val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _)
val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _)
val maxMem = storageStatusList.map(_.maxMem).sum
val remainingMem = storageStatusList.map(_.memRemaining).sum
(maxMem - remainingMem) / 1024 / 1024
}
})

metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed_MB"), new Gauge[Long] {
override def getValue: Long = {
val storageStatusList = blockManager.master.getStorageStatus
val diskSpaceUsed = storageStatusList
.flatMap(_.blocks.values.map(_.diskSize))
.reduceOption(_ + _)
.getOrElse(0L)

val diskSpaceUsed = storageStatusList.map(_.diskUsed).sum
diskSpaceUsed / 1024 / 1024
}
})
Expand Down
2 changes: 2 additions & 0 deletions core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,8 @@ class RDDInfo(
var diskSize = 0L
var tachyonSize = 0L

def isCached: Boolean = (memSize + diskSize + tachyonSize > 0) && numCachedPartitions > 0

override def toString = {
import Utils.bytesToString
("RDD \"%s\" (%d) StorageLevel: %s; CachedPartitions: %d; TotalPartitions: %d; " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,13 +35,12 @@ class StorageStatusListener extends SparkListener {

/** Update storage status list to reflect updated block statuses */
private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) {
val filteredStatus = executorIdToStorageStatus.get(execId)
filteredStatus.foreach { storageStatus =>
executorIdToStorageStatus.get(execId).foreach { storageStatus =>
updatedBlocks.foreach { case (blockId, updatedStatus) =>
if (updatedStatus.storageLevel == StorageLevel.NONE) {
storageStatus.blocks.remove(blockId)
storageStatus.removeBlock(blockId)
} else {
storageStatus.blocks(blockId) = updatedStatus
storageStatus.updateBlock(blockId, updatedStatus)
}
}
}
Expand All @@ -50,9 +49,8 @@ class StorageStatusListener extends SparkListener {
/** Update storage status list to reflect the removal of an RDD from the cache */
private def updateStorageStatus(unpersistedRDDId: Int) {
storageStatusList.foreach { storageStatus =>
val unpersistedBlocksIds = storageStatus.rddBlocks.keys.filter(_.rddId == unpersistedRDDId)
unpersistedBlocksIds.foreach { blockId =>
storageStatus.blocks.remove(blockId)
storageStatus.rddBlocksById(unpersistedRDDId).foreach { case (blockId, _) =>
storageStatus.removeBlock(blockId)
}
}
}
Expand Down
Loading

0 comments on commit d934801

Please sign in to comment.