From 2794daacb8bc300dd35c991a1bd2f074b223cfc1 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 12 Aug 2016 10:51:12 +0800 Subject: [PATCH 01/11] Expose on-heap and off-heap memory usage in UI, REST API and SparkListener --- .../apache/spark/ui/static/executorspage.js | 6 +- .../spark/scheduler/SparkListener.scala | 9 ++- .../org/apache/spark/status/api/v1/api.scala | 6 +- .../apache/spark/storage/BlockManager.scala | 9 +-- .../spark/storage/BlockManagerMaster.scala | 7 +- .../storage/BlockManagerMasterEndpoint.scala | 21 +++--- .../spark/storage/BlockManagerMessages.scala | 3 +- .../spark/storage/BlockManagerSource.scala | 66 +++++++++--------- .../spark/storage/StorageStatusListener.scala | 5 +- .../apache/spark/storage/StorageUtils.scala | 66 ++++++++++++++---- .../apache/spark/ui/exec/ExecutorsPage.scala | 13 +++- .../org/apache/spark/util/JsonProtocol.scala | 8 ++- .../executor_list_json_expectation.json | 6 +- .../apache/spark/storage/StorageSuite.scala | 67 +++++++++++++++++-- 14 files changed, 216 insertions(+), 76 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 7dbfe32de903a..e43fd25676953 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -378,7 +378,11 @@ $(document).ready(function () { {data: 'rddBlocks'}, { data: function (row, type) { - return type === 'display' ? (formatBytes(row.memoryUsed, type) + ' / ' + formatBytes(row.maxMemory, type)) : row.memoryUsed; + return '
' + + (type === 'display' ? (formatBytes(row.onHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOnHeapMemory, type)) : row.maxOnHeapMemory) + ' (On Heap)' + + '
' + + (type === 'display' ? (formatBytes(row.offHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOffHeapMemory, type)) : row.maxOffHeapMemory) + ' (Off Heap)' + + '
' ; } }, {data: 'diskUsed', render: formatBytes}, diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 4331addb44172..bc2e530716686 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -87,8 +87,13 @@ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(S extends SparkListenerEvent @DeveloperApi -case class SparkListenerBlockManagerAdded(time: Long, blockManagerId: BlockManagerId, maxMem: Long) - extends SparkListenerEvent +case class SparkListenerBlockManagerAdded( + time: Long, + blockManagerId: BlockManagerId, + maxMem: Long, + maxOnHeapMem: Option[Long] = None, + maxOffHeapMem: Option[Long] = None) extends SparkListenerEvent { +} @DeveloperApi case class SparkListenerBlockManagerRemoved(time: Long, blockManagerId: BlockManagerId) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 5b9227350edaa..8c0035ee6e9da 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -75,7 +75,11 @@ class ExecutorSummary private[spark]( val totalShuffleWrite: Long, val isBlacklisted: Boolean, val maxMemory: Long, - val executorLogs: Map[String, String]) + val executorLogs: Map[String, String], + val onHeapMemoryUsed: Long, + val offHeapMemoryUsed: Long, + val maxOnHeapMemory: Long, + val maxOffHeapMemory: Long) class JobData private[spark]( val jobId: Int, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 245d94ac4f8b1..07a4b59198835 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -101,8 +101,8 @@ private[spark] class BlockManager( // However, since we use this only for reporting and logging, what we actually want here is // the absolute maximum value that `maxMemory` can ever possibly reach. We may need // to revisit whether reporting this value as the "max" is intuitive to the user. - private val maxMemory = - memoryManager.maxOnHeapStorageMemory + memoryManager.maxOffHeapStorageMemory + private val maxOnHeapMemory = memoryManager.maxOnHeapStorageMemory + private val maxOffHeapMemory = memoryManager.maxOffHeapStorageMemory // Port used by the external shuffle service. In Yarn mode, this may be already be // set through the Hadoop configuration as the server is launched in the Yarn NM. @@ -180,7 +180,8 @@ private[spark] class BlockManager( val idFromMaster = master.registerBlockManager( id, - maxMemory, + maxOffHeapMemory, + maxOffHeapMemory, slaveEndpoint) blockManagerId = if (idFromMaster != null) idFromMaster else id @@ -258,7 +259,7 @@ private[spark] class BlockManager( def reregister(): Unit = { // TODO: We might need to rate limit re-registering. logInfo(s"BlockManager $blockManagerId re-registering with master") - master.registerBlockManager(blockManagerId, maxMemory, slaveEndpoint) + master.registerBlockManager(blockManagerId, maxOnHeapMemory, maxOffHeapMemory, slaveEndpoint) reportAllBlocks() } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 3ca690db9e79f..d83e02f8d64f2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -57,12 +57,13 @@ class BlockManagerMaster( */ def registerBlockManager( blockManagerId: BlockManagerId, - maxMemSize: Long, + maxOnHeapMemSize: Long, + maxOffHeapMemSize: Long, slaveEndpoint: RpcEndpointRef): BlockManagerId = { logInfo(s"Registering BlockManager $blockManagerId") val updatedId = driverEndpoint.askSync[BlockManagerId]( - RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint)) - logInfo(s"Registered BlockManager $updatedId") + RegisterBlockManager(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) + logInfo(s"Registered BlockManager $blockManagerId") updatedId } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 84c04d22600ad..9c2e8e10ce2d3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -71,8 +71,8 @@ class BlockManagerMasterEndpoint( logInfo("BlockManagerMasterEndpoint up") override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint) => - context.reply(register(blockManagerId, maxMemSize, slaveEndpoint)) + case RegisterBlockManager(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) => + context.reply(register(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) case _updateBlockInfo @ UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => @@ -276,7 +276,7 @@ class BlockManagerMasterEndpoint( private def storageStatus: Array[StorageStatus] = { blockManagerInfo.map { case (blockManagerId, info) => - new StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala) + new StorageStatus(blockManagerId, info.maxOnHeapMem, info.maxOffHeapMem, info.blocks.asScala) }.toArray } @@ -338,7 +338,8 @@ class BlockManagerMasterEndpoint( */ private def register( idWithoutTopologyInfo: BlockManagerId, - maxMemSize: Long, + maxOnHeapMemSize: Long, + maxOffHeapMemSize: Long, slaveEndpoint: RpcEndpointRef): BlockManagerId = { // the dummy id is not expected to contain the topology information. // we get that info here and respond back with a more fleshed out block manager id @@ -359,14 +360,15 @@ class BlockManagerMasterEndpoint( case None => } logInfo("Registering block manager %s with %s RAM, %s".format( - id.hostPort, Utils.bytesToString(maxMemSize), id)) + id.hostPort, Utils.bytesToString(maxOnHeapMemSize + maxOffHeapMemSize), id)) blockManagerIdByExecutor(id.executorId) = id blockManagerInfo(id) = new BlockManagerInfo( - id, System.currentTimeMillis(), maxMemSize, slaveEndpoint) + id, System.currentTimeMillis(), maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) } - listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxMemSize)) + listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxOnHeapMemSize + maxOffHeapMemSize, + Some(maxOnHeapMemSize), Some(maxOffHeapMemSize))) id } @@ -464,10 +466,13 @@ object BlockStatus { private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, - val maxMem: Long, + val maxOnHeapMem: Long, + val maxOffHeapMem: Long, val slaveEndpoint: RpcEndpointRef) extends Logging { + val maxMem = maxOnHeapMem + maxOffHeapMem + private var _lastSeenMs: Long = timeMs private var _remainingMem: Long = maxMem diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 0aea438e7f473..0c0ff144596ac 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -58,7 +58,8 @@ private[spark] object BlockManagerMessages { case class RegisterBlockManager( blockManagerId: BlockManagerId, - maxMemSize: Long, + maxOnHeapMemSize: Long, + maxOffHeapMemSize: Long, sender: RpcEndpointRef) extends ToBlockManagerMaster diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index c5ba9af3e2658..32bd42d4d2314 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -26,35 +26,39 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) override val metricRegistry = new MetricRegistry() override val sourceName = "BlockManager" - metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { - override def getValue: Long = { - val storageStatusList = blockManager.master.getStorageStatus - 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).sum - remainingMem / 1024 / 1024 - } - }) - - metricRegistry.register(MetricRegistry.name("memory", "memUsed_MB"), new Gauge[Long] { - override def getValue: Long = { - val storageStatusList = blockManager.master.getStorageStatus - val memUsed = storageStatusList.map(_.memUsed).sum - memUsed / 1024 / 1024 - } - }) - - metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed_MB"), new Gauge[Long] { - override def getValue: Long = { - val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.map(_.diskUsed).sum - diskSpaceUsed / 1024 / 1024 - } - }) + private def registerGauge[T](name: String, f: BlockManagerMaster => T): Unit = { + metricRegistry.register(name, new Gauge[T] { + override def getValue: T = f(blockManager.master) + }) + } + + registerGauge(MetricRegistry.name("memory", "maxMem_MB"), + _.getStorageStatus.map(_.maxMem).sum / 1024 / 1024) + + registerGauge(MetricRegistry.name("memory", "maxOnHeapMem_MB"), + _.getStorageStatus.map(_.maxOnHeapMem).sum / 1024 / 1024) + + registerGauge(MetricRegistry.name("memory", "maxOffHeapMem_MB"), + _.getStorageStatus.map(_.maxOffHeapMem).sum / 1024 / 1024) + + registerGauge(MetricRegistry.name("memory", "remainingMem_MB"), + _.getStorageStatus.map(_.memRemaining).sum / 1024 / 1024) + + registerGauge(MetricRegistry.name("memory", "remainingOnHeapMem_MB"), + _.getStorageStatus.map(_.onHeapMemRemaining).sum / 1024 / 1024) + + registerGauge(MetricRegistry.name("memory", "remainingOffHeapMem_MB"), + _.getStorageStatus.map(_.offHeapMemRemaining).sum / 1024 / 1024) + + registerGauge(MetricRegistry.name("memory", "memUsed_MB"), + _.getStorageStatus.map(_.memUsed).sum / 1024 / 1024) + + registerGauge(MetricRegistry.name("memory", "onHeapMemUsed_MB"), + _.getStorageStatus.map(_.onHeapMemUsed).sum / 1024 / 1024) + + registerGauge(MetricRegistry.name("memory", "offHeapMemUsed_MB"), + _.getStorageStatus.map(_.offHeapMemUsed).sum / 1024 / 1024) + + registerGauge(MetricRegistry.name("disk", "diskSpaceUsed_MB"), + _.getStorageStatus.map(_.diskUsed).sum / 1024 / 1024) } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 798658a15b797..20d56fa125f91 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -74,8 +74,9 @@ class StorageStatusListener(conf: SparkConf) extends SparkListener { synchronized { val blockManagerId = blockManagerAdded.blockManagerId val executorId = blockManagerId.executorId - val maxMem = blockManagerAdded.maxMem - val storageStatus = new StorageStatus(blockManagerId, maxMem) + val maxOnHeapMem = blockManagerAdded.maxOnHeapMem.getOrElse(blockManagerAdded.maxMem) + val maxOffHeapMem = blockManagerAdded.maxOffHeapMem.getOrElse(0L) + val storageStatus = new StorageStatus(blockManagerId, maxOnHeapMem, maxOffHeapMem) executorIdToStorageStatus(executorId) = storageStatus // Try to remove the dead storage status if same executor register the block manager twice. diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 5efdd23f79a21..9b2f9543a23a5 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -35,7 +35,10 @@ import org.apache.spark.internal.Logging * class cannot mutate the source of the information. Accesses are not thread-safe. */ @DeveloperApi -class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { +class StorageStatus( + val blockManagerId: BlockManagerId, + val maxOnHeapMem: Long, + val maxOffHeapMem: Long) { /** * Internal representation of the blocks stored in this block manager. @@ -51,7 +54,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { * * As with the block maps, we store the storage information separately for RDD blocks and * non-RDD blocks for the same reason. In particular, RDD storage information is stored - * in a map indexed by the RDD ID to the following 4-tuple: + * in a map indexed by the RDD ID to the following 3-tuple: * * (memory size, disk size, storage level) * @@ -60,11 +63,17 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { * non-RDD blocks contains only the first 3 fields (in the same order). */ private val _rddStorageInfo = new mutable.HashMap[Int, (Long, Long, StorageLevel)] - private var _nonRddStorageInfo: (Long, Long) = (0L, 0L) + + // On-heap memory, off-heap memory and disk usage of non rdd storage + private var _nonRddStorageInfo: (Long, Long, Long) = (0L, 0L, 0L) /** Create a storage status with an initial set of blocks, leaving the source unmodified. */ - def this(bmid: BlockManagerId, maxMem: Long, initialBlocks: Map[BlockId, BlockStatus]) { - this(bmid, maxMem) + def this( + bmid: BlockManagerId, + maxOnHeapMem: Long, + maxOffHeapMem: Long, + initialBlocks: Map[BlockId, BlockStatus]) { + this(bmid, maxOnHeapMem, maxOffHeapMem) initialBlocks.foreach { case (bid, bstatus) => addBlock(bid, bstatus) } } @@ -176,17 +185,42 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { */ def numRddBlocksById(rddId: Int): Int = _rddBlocks.get(rddId).map(_.size).getOrElse(0) + /** Return the max memory can be used by this block manager. */ + def maxMem: Long = maxOnHeapMem + maxOffHeapMem + /** Return the memory remaining in this block manager. */ - def memRemaining: Long = maxMem - memUsed + def memRemaining: Long = onHeapMemRemaining + offHeapMemRemaining + + /** Return the memory used by caching RDDs */ + def cacheSize: Long = onHeapCacheSize + offHeapCacheSize /** Return the memory used by this block manager. */ - def memUsed: Long = _nonRddStorageInfo._1 + cacheSize + def memUsed: Long = onHeapMemUsed + offHeapMemUsed - /** Return the memory used by caching RDDs */ - def cacheSize: Long = _rddBlocks.keys.toSeq.map(memUsedByRdd).sum + /** Return the on-heap memory remaining in this block manager. */ + def onHeapMemRemaining: Long = maxOnHeapMem - onHeapMemUsed + + /** Return the off-heap memory remaining in this block manager. */ + def offHeapMemRemaining: Long = maxOffHeapMem - offHeapMemUsed + + /** Return the on-heap memory used by this block manager. */ + def onHeapMemUsed: Long = _nonRddStorageInfo._1 + onHeapCacheSize + + /** Return the off-heap memory used by this block manager. */ + def offHeapMemUsed: Long = _nonRddStorageInfo._2 + offHeapCacheSize + + /** Return the memory used by on-heap caching RDDs */ + def onHeapCacheSize: Long = { + _rddStorageInfo.filter(!_._2._3.useOffHeap).map(_._2._1).sum + } + + /** Return the memory used by off-heap caching RDDs */ + def offHeapCacheSize: Long = { + _rddStorageInfo.filter(_._2._3.useOffHeap).map(_._2._1).sum + } /** Return the disk space used by this block manager. */ - def diskUsed: Long = _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum + def diskUsed: Long = _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum /** Return the memory used by the given RDD in this block manager in O(1) time. */ def memUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._1).getOrElse(0L) @@ -212,8 +246,10 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { _rddStorageInfo.get(rddId) .map { case (mem, disk, _) => (mem, disk) } .getOrElse((0L, 0L)) - case _ => - _nonRddStorageInfo + case _ if !level.useOffHeap => + (_nonRddStorageInfo._1, _nonRddStorageInfo._3) + case _ if level.useOffHeap => + (_nonRddStorageInfo._2, _nonRddStorageInfo._3) } val newMem = math.max(oldMem + changeInMem, 0L) val newDisk = math.max(oldDisk + changeInDisk, 0L) @@ -228,7 +264,11 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { _rddStorageInfo(rddId) = (newMem, newDisk, level) } case _ => - _nonRddStorageInfo = (newMem, newDisk) + _nonRddStorageInfo = if (!level.useOffHeap) { + (newMem, _nonRddStorageInfo._2, newDisk) + } else { + (_nonRddStorageInfo._1, newMem, newDisk) + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 2d1691e55c428..9f55f94675f35 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -40,7 +40,8 @@ private[ui] case class ExecutorSummaryInfo( totalShuffleRead: Long, totalShuffleWrite: Long, isBlacklisted: Int, - maxMemory: Long, + maxOnHeapMem: Long, + maxOffHeapMem: Long, executorLogs: Map[String, String]) @@ -81,6 +82,10 @@ private[spark] object ExecutorsPage { val rddBlocks = status.numBlocks val memUsed = status.memUsed val maxMem = status.maxMem + val onHeapMemUsed = status.onHeapMemUsed + val offHeapMemUsed = status.offHeapMemUsed + val maxOnHeapMem = status.maxOnHeapMem + val maxOffHeapMem = status.maxOffHeapMem val diskUsed = status.diskUsed val taskSummary = listener.executorToTaskSummary.getOrElse(execId, ExecutorTaskSummary(execId)) @@ -104,7 +109,11 @@ private[spark] object ExecutorsPage { taskSummary.shuffleWrite, taskSummary.isBlacklisted, maxMem, - taskSummary.executorLogs + taskSummary.executorLogs, + onHeapMemUsed, + offHeapMemUsed, + maxOnHeapMem, + maxOffHeapMem ) } } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 2cb88919c8c83..43dcb10aa6167 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -182,7 +182,9 @@ private[spark] object JsonProtocol { ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.blockManagerAdded) ~ ("Block Manager ID" -> blockManagerId) ~ ("Maximum Memory" -> blockManagerAdded.maxMem) ~ - ("Timestamp" -> blockManagerAdded.time) + ("Timestamp" -> blockManagerAdded.time) ~ + ("Maximum Onheap Memory" -> blockManagerAdded.maxOnHeapMem) ~ + ("Maximum Offheap Memory" -> blockManagerAdded.maxOffHeapMem) } def blockManagerRemovedToJson(blockManagerRemoved: SparkListenerBlockManagerRemoved): JValue = { @@ -605,7 +607,9 @@ private[spark] object JsonProtocol { val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") val maxMem = (json \ "Maximum Memory").extract[Long] val time = Utils.jsonOption(json \ "Timestamp").map(_.extract[Long]).getOrElse(-1L) - SparkListenerBlockManagerAdded(time, blockManagerId, maxMem) + val maxOnHeapMem = Utils.jsonOption(json \ "Maximum Onheap Memory").map(_.extract[Long]) + val maxOffHeapMem = Utils.jsonOption(json \ "Maximum Offheap Memory").map(_.extract[Long]) + SparkListenerBlockManagerAdded(time, blockManagerId, maxMem, maxOnHeapMem, maxOffHeapMem) } def blockManagerRemovedFromJson(json: JValue): SparkListenerBlockManagerRemoved = { diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index 6b9f29e1a230e..f379151f91e0b 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -18,5 +18,9 @@ "totalShuffleWrite" : 13180, "isBlacklisted" : false, "maxMemory" : 278302556, - "executorLogs" : { } + "executorLogs" : { }, + "onHeapMemoryUsed": 0, + "offHeapMemoryUsed": 0, + "maxOnHeapMemory": 278302556, + "maxOffHeapMemory": 0 } ] diff --git a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala index e5733aebf607c..0ea0cd5f5557d 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -27,7 +27,7 @@ class StorageSuite extends SparkFunSuite { // For testing add, update, and remove (for non-RDD blocks) private def storageStatus1: StorageStatus = { - val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L) + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, 0L) assert(status.blocks.isEmpty) assert(status.rddBlocks.isEmpty) assert(status.memUsed === 0L) @@ -74,7 +74,7 @@ class StorageSuite extends SparkFunSuite { // For testing add, update, remove, get, and contains etc. for both RDD and non-RDD blocks private def storageStatus2: StorageStatus = { - val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L) + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, 0L) assert(status.rddBlocks.isEmpty) status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L)) status.addBlock(TestBlockId("man"), BlockStatus(memAndDisk, 10L, 20L)) @@ -252,9 +252,9 @@ class StorageSuite extends SparkFunSuite { // For testing StorageUtils.updateRddInfo and StorageUtils.getRddBlockLocations private def stockStorageStatuses: Seq[StorageStatus] = { - val status1 = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L) - val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2), 2000L) - val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3), 3000L) + val status1 = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, 0L) + val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2), 2000L, 0L) + val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3), 3000L, 0L) status1.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L)) status1.addBlock(RDDBlockId(0, 1), BlockStatus(memAndDisk, 1L, 2L)) status2.addBlock(RDDBlockId(0, 2), BlockStatus(memAndDisk, 1L, 2L)) @@ -332,4 +332,61 @@ class StorageSuite extends SparkFunSuite { assert(blockLocations1(RDDBlockId(1, 2)) === Seq("cat:3")) } + private val offheap = StorageLevel.OFF_HEAP + // For testing add, update, remove, get, and contains etc. for both RDD and non-RDD onheap + // and offheap blocks + private def storageStatus3: StorageStatus = { + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, 1000L) + assert(status.rddBlocks.isEmpty) + status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(TestBlockId("man"), BlockStatus(offheap, 10L, 0L)) + status.addBlock(RDDBlockId(0, 0), BlockStatus(offheap, 10L, 0L)) + status.addBlock(RDDBlockId(1, 1), BlockStatus(offheap, 100L, 0L)) + status.addBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(RDDBlockId(2, 3), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(RDDBlockId(2, 4), BlockStatus(memAndDisk, 10L, 40L)) + status + } + + test("storage memUsed, diskUsed with on-heap and off-heap blocks") { + val status = storageStatus3 + def actualMemUsed: Long = status.blocks.values.map(_.memSize).sum + def actualDiskUsed: Long = status.blocks.values.map(_.diskSize).sum + + def actualOnHeapMemUsed: Long = + status.blocks.values.filter(!_.storageLevel.useOffHeap).map(_.memSize).sum + def actualOffHeapMemUsed: Long = + status.blocks.values.filter(_.storageLevel.useOffHeap).map(_.memSize).sum + + assert(status.maxMem === status.maxOnHeapMem + status.maxOffHeapMem) + + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.onHeapMemUsed === actualOnHeapMemUsed) + assert(status.offHeapMemUsed === actualOffHeapMemUsed) + + assert(status.memRemaining === status.maxMem - actualMemUsed) + assert(status.onHeapMemRemaining === status.maxOnHeapMem - actualOnHeapMemUsed) + assert(status.offHeapMemRemaining === status.maxOffHeapMem - actualOffHeapMemUsed) + + status.addBlock(TestBlockId("wire"), BlockStatus(memAndDisk, 400L, 500L)) + status.addBlock(RDDBlockId(25, 25), BlockStatus(memAndDisk, 40L, 50L)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + + status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 4L, 5L)) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(offheap, 4L, 0L)) + status.updateBlock(RDDBlockId(1, 1), BlockStatus(offheap, 4L, 0L)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.onHeapMemUsed === actualOnHeapMemUsed) + assert(status.offHeapMemUsed === actualOffHeapMemUsed) + + status.removeBlock(TestBlockId("fire")) + status.removeBlock(TestBlockId("man")) + status.removeBlock(RDDBlockId(2, 2)) + status.removeBlock(RDDBlockId(2, 3)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + } } From 5f1fb3cc1c31d23f00051f7b1d4d6b49f7189c3e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 12 Aug 2016 12:00:16 +0800 Subject: [PATCH 02/11] Add mima excludes --- project/MimaExcludes.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 2e3f9f2d0f3ac..4bbaa4569be10 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -100,7 +100,15 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toDenseMatrix"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.toSparseMatrix"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Matrix.getSizeInBytes") - ) + ) ++ Seq( + // [SPARK-17019] Expose on-heap and off-heap memory usage in various places + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.this"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.apply"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.StorageStatus.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.this") + ) // Exclude rules for 2.1.x lazy val v21excludes = v20excludes ++ { @@ -178,7 +186,8 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.startOffset"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.endOffset"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryException.query") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryException.query"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.Dataset.select") ) } From 572cba612c0645e12be9968a1cff8d449bcae395 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 25 Aug 2016 15:27:18 +0800 Subject: [PATCH 03/11] Address the comment to seperate on heap and off heap memory storage in different column --- .../ui/static/executorspage-template.html | 13 +++- .../apache/spark/ui/static/executorspage.js | 66 ++++++++++++------- 2 files changed, 54 insertions(+), 25 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 4e83d6d564986..fbfe9d4734320 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -24,7 +24,10 @@

Summary

RDD Blocks Storage Memory + title="Memory used / total available memory for on heap storage of data like RDD partitions cached in memory. "> On Heap Storage Memory + + Off Heap Storage Memory Disk Used Cores @@ -71,8 +74,12 @@

Executors

RDD Blocks - Storage Memory + title="Memory used / total available memory for on heap storage of data like RDD partitions cached in memory."> + On Heap Storage Memory + + + Off Heap Storage Memory Disk Used Cores Active Tasks diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index e43fd25676953..8844273c971d7 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -188,8 +188,10 @@ $(document).ready(function () { var summary = []; var allExecCnt = 0; var allRDDBlocks = 0; - var allMemoryUsed = 0; - var allMaxMemory = 0; + var allOnHeapMemoryUsed = 0; + var allOnHeapMaxMemory = 0; + var allOffHeapMemoryUsed = 0; + var allOffHeapMaxMemory = 0; var allDiskUsed = 0; var allTotalCores = 0; var allMaxTasks = 0; @@ -206,8 +208,10 @@ $(document).ready(function () { var activeExecCnt = 0; var activeRDDBlocks = 0; - var activeMemoryUsed = 0; - var activeMaxMemory = 0; + var activeOnHeapMemoryUsed = 0; + var activeOnHeapMaxMemory = 0; + var activeOffHeapMemoryUsed = 0; + var activeOffHeapMaxMemory = 0; var activeDiskUsed = 0; var activeTotalCores = 0; var activeMaxTasks = 0; @@ -224,8 +228,10 @@ $(document).ready(function () { var deadExecCnt = 0; var deadRDDBlocks = 0; - var deadMemoryUsed = 0; - var deadMaxMemory = 0; + var deadOnHeapMemoryUsed = 0; + var deadOnHeapMaxMemory = 0; + var deadOffHeapMemoryUsed = 0; + var deadOffHeapMaxMemory = 0; var deadDiskUsed = 0; var deadTotalCores = 0; var deadMaxTasks = 0; @@ -243,8 +249,10 @@ $(document).ready(function () { response.forEach(function (exec) { allExecCnt += 1; allRDDBlocks += exec.rddBlocks; - allMemoryUsed += exec.memoryUsed; - allMaxMemory += exec.maxMemory; + allOnHeapMemoryUsed += exec.onHeapMemoryUsed; + allOnHeapMaxMemory += exec.maxOnHeapMemory; + allOffHeapMemoryUsed += exec.offHeapMemoryUsed; + allOffHeapMaxMemory += exec.maxOffHeapMemory; allDiskUsed += exec.diskUsed; allTotalCores += exec.totalCores; allMaxTasks += exec.maxTasks; @@ -261,8 +269,10 @@ $(document).ready(function () { if (exec.isActive) { activeExecCnt += 1; activeRDDBlocks += exec.rddBlocks; - activeMemoryUsed += exec.memoryUsed; - activeMaxMemory += exec.maxMemory; + activeOnHeapMemoryUsed += exec.onHeapMemoryUsed; + activeOnHeapMaxMemory += exec.maxOnHeapMemory; + activeOffHeapMemoryUsed += exec.offHeapMemoryUsed; + activeOffHeapMaxMemory += exec.maxOffHeapMemory; activeDiskUsed += exec.diskUsed; activeTotalCores += exec.totalCores; activeMaxTasks += exec.maxTasks; @@ -300,8 +310,10 @@ $(document).ready(function () { var totalSummary = { "execCnt": ( "Total(" + allExecCnt + ")"), "allRDDBlocks": allRDDBlocks, - "allMemoryUsed": allMemoryUsed, - "allMaxMemory": allMaxMemory, + "allOnHeapMemoryUsed": allOnHeapMemoryUsed, + "allOnHeapMaxMemory": allOnHeapMaxMemory, + "allOffHeapMemoryUsed": allOffHeapMemoryUsed, + "allOffHeapMaxMemory": allOffHeapMaxMemory, "allDiskUsed": allDiskUsed, "allTotalCores": allTotalCores, "allMaxTasks": allMaxTasks, @@ -319,8 +331,10 @@ $(document).ready(function () { var activeSummary = { "execCnt": ( "Active(" + activeExecCnt + ")"), "allRDDBlocks": activeRDDBlocks, - "allMemoryUsed": activeMemoryUsed, - "allMaxMemory": activeMaxMemory, + "allOnHeapMemoryUsed": activeOnHeapMemoryUsed, + "allOnHeapMaxMemory": activeOnHeapMaxMemory, + "allOffHeapMemoryUsed": activeOffHeapMemoryUsed, + "allOffHeapMaxMemory": activeOffHeapMaxMemory, "allDiskUsed": activeDiskUsed, "allTotalCores": activeTotalCores, "allMaxTasks": activeMaxTasks, @@ -338,8 +352,10 @@ $(document).ready(function () { var deadSummary = { "execCnt": ( "Dead(" + deadExecCnt + ")" ), "allRDDBlocks": deadRDDBlocks, - "allMemoryUsed": deadMemoryUsed, - "allMaxMemory": deadMaxMemory, + "allOnHeapMemoryUsed": deadOnHeapMemoryUsed, + "allOnHeapMaxMemory": deadOnHeapMaxMemory, + "allOffHeapMemoryUsed": deadOffHeapMemoryUsed, + "allOffHeapMaxMemory": deadOffHeapMaxMemory, "allDiskUsed": deadDiskUsed, "allTotalCores": deadTotalCores, "allMaxTasks": deadMaxTasks, @@ -378,11 +394,12 @@ $(document).ready(function () { {data: 'rddBlocks'}, { data: function (row, type) { - return '
' + - (type === 'display' ? (formatBytes(row.onHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOnHeapMemory, type)) : row.maxOnHeapMemory) + ' (On Heap)' + - '
' + - (type === 'display' ? (formatBytes(row.offHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOffHeapMemory, type)) : row.maxOffHeapMemory) + ' (Off Heap)' + - '
' ; + return type === 'display' ? (formatBytes(row.onHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOnHeapMemory, type)) : row.maxOnHeapMemory + } + }, + { + data: function (row, type) { + return type === 'display' ? (formatBytes(row.offHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOffHeapMemory, type)) : row.maxOffHeapMemory } }, {data: 'diskUsed', render: formatBytes}, @@ -454,7 +471,12 @@ $(document).ready(function () { {data: 'allRDDBlocks'}, { data: function (row, type) { - return type === 'display' ? (formatBytes(row.allMemoryUsed, type) + ' / ' + formatBytes(row.allMaxMemory, type)) : row.allMemoryUsed; + return type === 'display' ? (formatBytes(row.allOnHeapMemoryUsed, type) + ' / ' + formatBytes(row.allOnHeapMaxMemory, type)) : row.allOnHeapMemoryUsed; + } + }, + { + data: function (row, type) { + return type === 'display' ? (formatBytes(row.allOffHeapMemoryUsed, type) + ' / ' + formatBytes(row.allOffHeapMaxMemory, type)) : row.allOffHeapMemoryUsed; } }, {data: 'allDiskUsed', render: formatBytes}, From f02b5ff64ca841b22a70de75edf221d0b1650586 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 20 Mar 2017 19:49:28 +0800 Subject: [PATCH 04/11] Change to display on heap and off heap memory usage in tooltip Change-Id: Ib66000929019734aef7a38a17bdfe140239fb1c0 --- .../ui/static/executorspage-template.html | 13 ++----- .../apache/spark/ui/static/executorspage.js | 37 +++++++++++++------ .../spark/storage/BlockManagerSource.scala | 26 ++++++------- project/MimaExcludes.scala | 3 +- 4 files changed, 42 insertions(+), 37 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index fbfe9d4734320..4e5ce6f945de3 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -24,10 +24,7 @@

Summary

RDD Blocks On Heap Storage Memory - - Off Heap Storage Memory + title="Memory used / total available memory for storage of data like RDD partitions cached in memory."> Storage Memory Disk Used Cores @@ -74,12 +71,8 @@

Executors

RDD Blocks - On Heap Storage Memory - - - Off Heap Storage Memory + title="Memory used / total available memory for storage of data like RDD partitions cached in memory."> + Storage Memory Disk Used Cores Active Tasks diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 8844273c971d7..b8b5c7ddf51ba 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -172,6 +172,15 @@ function totalDurationColor(totalGCTime, totalDuration) { return (totalGCTime > GCTimePercent * totalDuration) ? "white" : "black"; } +function memoryUsageTooltip(onHeap, maxOnHeap, offHeap, maxOffHeap, onHeapSum, offHeapSum, type) { + return ("" + + formatBytes(onHeapSum, type) + " / " + formatBytes(offHeapSum, type) + ""); +} + $(document).ready(function () { $.extend($.fn.dataTable.defaults, { stateSave: true, @@ -394,12 +403,14 @@ $(document).ready(function () { {data: 'rddBlocks'}, { data: function (row, type) { - return type === 'display' ? (formatBytes(row.onHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOnHeapMemory, type)) : row.maxOnHeapMemory - } - }, - { - data: function (row, type) { - return type === 'display' ? (formatBytes(row.offHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOffHeapMemory, type)) : row.maxOffHeapMemory + if (type !== 'display') + return row.maxOnHeapMemory + row.maxOffHeapMemory; + else + var memoryUsed = row.onHeapMemoryUsed + row.offHeapMemoryUsed + var maxMemory = row.maxOnHeapMemory + row.maxOffHeapMemory + return memoryUsageTooltip(row.onHeapMemoryUsed, row.maxOnHeapMemory, + row.offHeapMemoryUsed, row.maxOffHeapMemory, memoryUsed, maxMemory, type); + } }, {data: 'diskUsed', render: formatBytes}, @@ -471,12 +482,14 @@ $(document).ready(function () { {data: 'allRDDBlocks'}, { data: function (row, type) { - return type === 'display' ? (formatBytes(row.allOnHeapMemoryUsed, type) + ' / ' + formatBytes(row.allOnHeapMaxMemory, type)) : row.allOnHeapMemoryUsed; - } - }, - { - data: function (row, type) { - return type === 'display' ? (formatBytes(row.allOffHeapMemoryUsed, type) + ' / ' + formatBytes(row.allOffHeapMaxMemory, type)) : row.allOffHeapMemoryUsed; + if (type !== 'display') + return row.allOnHeapMemoryUsed + row.allOffHeapMemoryUsed; + else + var memoryUsed = row.allOnHeapMemoryUsed + row.allOffHeapMemoryUsed + var maxMemory = row.allOnHeapMaxMemory + row.allOffHeapMaxMemory + return memoryUsageTooltip(row.allOnHeapMemoryUsed, row.allOnHeapMaxMemory, + row.allOffHeapMemoryUsed, row.allOffHeapMaxMemory, memoryUsed, maxMemory, type); + } }, {data: 'allDiskUsed', render: formatBytes}, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 32bd42d4d2314..c19b0f44d080a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -26,39 +26,39 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) override val metricRegistry = new MetricRegistry() override val sourceName = "BlockManager" - private def registerGauge[T](name: String, f: BlockManagerMaster => T): Unit = { - metricRegistry.register(name, new Gauge[T] { - override def getValue: T = f(blockManager.master) + private def registerGauge(name: String, f: BlockManagerMaster => Long): Unit = { + metricRegistry.register(name, new Gauge[Long] { + override def getValue: Long = f(blockManager.master) / 1024 / 1024 }) } registerGauge(MetricRegistry.name("memory", "maxMem_MB"), - _.getStorageStatus.map(_.maxMem).sum / 1024 / 1024) + _.getStorageStatus.map(_.maxMem).sum) registerGauge(MetricRegistry.name("memory", "maxOnHeapMem_MB"), - _.getStorageStatus.map(_.maxOnHeapMem).sum / 1024 / 1024) + _.getStorageStatus.map(_.maxOnHeapMem).sum) registerGauge(MetricRegistry.name("memory", "maxOffHeapMem_MB"), - _.getStorageStatus.map(_.maxOffHeapMem).sum / 1024 / 1024) + _.getStorageStatus.map(_.maxOffHeapMem).sum) registerGauge(MetricRegistry.name("memory", "remainingMem_MB"), - _.getStorageStatus.map(_.memRemaining).sum / 1024 / 1024) + _.getStorageStatus.map(_.memRemaining).sum) registerGauge(MetricRegistry.name("memory", "remainingOnHeapMem_MB"), - _.getStorageStatus.map(_.onHeapMemRemaining).sum / 1024 / 1024) + _.getStorageStatus.map(_.onHeapMemRemaining).sum) registerGauge(MetricRegistry.name("memory", "remainingOffHeapMem_MB"), - _.getStorageStatus.map(_.offHeapMemRemaining).sum / 1024 / 1024) + _.getStorageStatus.map(_.offHeapMemRemaining).sum) registerGauge(MetricRegistry.name("memory", "memUsed_MB"), - _.getStorageStatus.map(_.memUsed).sum / 1024 / 1024) + _.getStorageStatus.map(_.memUsed).sum) registerGauge(MetricRegistry.name("memory", "onHeapMemUsed_MB"), - _.getStorageStatus.map(_.onHeapMemUsed).sum / 1024 / 1024) + _.getStorageStatus.map(_.onHeapMemUsed).sum) registerGauge(MetricRegistry.name("memory", "offHeapMemUsed_MB"), - _.getStorageStatus.map(_.offHeapMemUsed).sum / 1024 / 1024) + _.getStorageStatus.map(_.offHeapMemUsed).sum) registerGauge(MetricRegistry.name("disk", "diskSpaceUsed_MB"), - _.getStorageStatus.map(_.diskUsed).sum / 1024 / 1024) + _.getStorageStatus.map(_.diskUsed).sum) } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4bbaa4569be10..e07a0a82c3560 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -186,8 +186,7 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.startOffset"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.endOffset"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryException.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryException.query"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.Dataset.select") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryException.query") ) } From 854116b4ca58ca2ac593a7cd992cfa6a4ed5e95f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 22 Mar 2017 10:59:36 +0800 Subject: [PATCH 05/11] Make additional column hidden by default Change-Id: I5a023ca93309444bdbe982712fd747fe97da5a00 --- .../ui/static/executorspage-template.html | 18 ++++- .../apache/spark/ui/static/executorspage.js | 78 +++++++++++++++---- .../org/apache/spark/ui/static/webui.css | 3 +- .../spark/status/api/v1/AllRDDResource.scala | 8 +- .../org/apache/spark/status/api/v1/api.scala | 6 +- .../apache/spark/storage/BlockManager.scala | 2 +- .../apache/spark/storage/StorageUtils.scala | 12 +-- .../apache/spark/ui/exec/ExecutorsPage.scala | 33 ++++++++ .../org/apache/spark/ui/storage/RDDPage.scala | 11 ++- ...xecutor_node_blacklisting_expectation.json | 30 +++++-- ...acklisting_unblacklisting_expectation.json | 30 +++++-- 11 files changed, 190 insertions(+), 41 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 4e5ce6f945de3..5c91304e49fd7 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -24,7 +24,15 @@

Summary

RDD Blocks Storage Memory + title="Memory used / total available memory for storage of data like RDD partitions cached in memory.">Storage Memory + + + On Heap Storage Memory + + + Off Heap Storage Memory Disk Used Cores @@ -73,6 +81,14 @@

Executors

Storage Memory + + + On Heap Storage Memory + + + Off Heap Storage Memory Disk Used Cores Active Tasks diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index b8b5c7ddf51ba..b8089a502219c 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -172,15 +172,6 @@ function totalDurationColor(totalGCTime, totalDuration) { return (totalGCTime > GCTimePercent * totalDuration) ? "white" : "black"; } -function memoryUsageTooltip(onHeap, maxOnHeap, offHeap, maxOffHeap, onHeapSum, offHeapSum, type) { - return ("" + - formatBytes(onHeapSum, type) + " / " + formatBytes(offHeapSum, type) + ""); -} - $(document).ready(function () { $.extend($.fn.dataTable.defaults, { stateSave: true, @@ -406,11 +397,37 @@ $(document).ready(function () { if (type !== 'display') return row.maxOnHeapMemory + row.maxOffHeapMemory; else - var memoryUsed = row.onHeapMemoryUsed + row.offHeapMemoryUsed - var maxMemory = row.maxOnHeapMemory + row.maxOffHeapMemory - return memoryUsageTooltip(row.onHeapMemoryUsed, row.maxOnHeapMemory, - row.offHeapMemoryUsed, row.maxOffHeapMemory, memoryUsed, maxMemory, type); + var memoryUsed = row.onHeapMemoryUsed + row.offHeapMemoryUsed; + var maxMemory = row.maxOnHeapMemory + row.maxOffHeapMemory; + return (formatBytes(memoryUsed, type) + ' / ' + + formatBytes(maxMemory, type)); + + } + }, + { + data: function (row, type) { + if (type !== 'display') + return row.maxOnHeapMemory; + else + return (formatBytes(row.onHeapMemoryUsed, type) + ' / ' + + formatBytes(row.maxOnHeapMemory, type)); + + }, + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + $(nTd).addClass('on_heap_memory') + } + }, + { + data: function (row, type) { + if (type !== 'display') + return row.maxOffHeapMemory; + else + return (formatBytes(row.offHeapMemoryUsed, type) + ' / ' + + formatBytes(row.maxOffHeapMemory, type)); + }, + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + $(nTd).addClass('off_heap_memory') } }, {data: 'diskUsed', render: formatBytes}, @@ -485,13 +502,40 @@ $(document).ready(function () { if (type !== 'display') return row.allOnHeapMemoryUsed + row.allOffHeapMemoryUsed; else - var memoryUsed = row.allOnHeapMemoryUsed + row.allOffHeapMemoryUsed - var maxMemory = row.allOnHeapMaxMemory + row.allOffHeapMaxMemory - return memoryUsageTooltip(row.allOnHeapMemoryUsed, row.allOnHeapMaxMemory, - row.allOffHeapMemoryUsed, row.allOffHeapMaxMemory, memoryUsed, maxMemory, type); + var memoryUsed = row.allOnHeapMemoryUsed + row.allOffHeapMemoryUsed; + var maxMemory = row.allOnHeapMaxMemory + row.allOffHeapMaxMemory; + return (formatBytes(memoryUsed, type) + ' / ' + + formatBytes(maxMemory, type)); } }, + { + data: function (row, type) { + if (type !== 'display') + return row.allOnHeapMemoryUsed; + else + return (formatBytes(row.allOnHeapMemoryUsed, type) + ' / ' + + formatBytes(row.allOnHeapMaxMemory, type)); + + }, + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + $(nTd).addClass('on_heap_memory') + } + }, + { + data: function (row, type) { + if (type !== 'display') + return row.allOffHeapMemoryUsed; + else + return (formatBytes(row.allOffHeapMemoryUsed, type) + ' / ' + + formatBytes(row.allOffHeapMaxMemory, type)); + + }, + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + $(nTd).addClass('off_heap_memory') + } + + }, {data: 'allDiskUsed', render: formatBytes}, {data: 'allTotalCores'}, { diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 319a719efaa79..935d9b1aec615 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -205,7 +205,8 @@ span.additional-metric-title { /* Hide all additional metrics by default. This is done here rather than using JavaScript to * avoid slow page loads for stage pages with large numbers (e.g., thousands) of tasks. */ .scheduler_delay, .deserialization_time, .fetch_wait_time, .shuffle_read_remote, -.serialization_time, .getting_result_time, .peak_execution_memory { +.serialization_time, .getting_result_time, .peak_execution_memory, +.on_heap_memory, .off_heap_memory { display: none; } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala index 5c03609e5e5e5..e43252c53cc82 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala @@ -70,7 +70,13 @@ private[spark] object AllRDDResource { address = status.blockManagerId.hostPort, memoryUsed = status.memUsedByRdd(rddId), memoryRemaining = status.memRemaining, - diskUsed = status.diskUsedByRdd(rddId) + diskUsed = status.diskUsedByRdd(rddId), + onHeapMemoryUsed = + if (!rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L, + offHeapMemoryUsed = + if (rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L, + onHeapMemoryRemaining = status.onHeapMemRemaining, + offHeapMemoryRemaining = status.offHeapMemRemaining ) } ) } else { None diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 8c0035ee6e9da..f4d58e38e360e 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -115,7 +115,11 @@ class RDDDataDistribution private[spark]( val address: String, val memoryUsed: Long, val memoryRemaining: Long, - val diskUsed: Long) + val diskUsed: Long, + val onHeapMemoryUsed: Long, + val offHeapMemoryUsed: Long, + val onHeapMemoryRemaining: Long, + val offHeapMemoryRemaining: Long) class RDDPartitionInfo private[spark]( val blockName: String, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 07a4b59198835..9f69e920c9ceb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -180,7 +180,7 @@ private[spark] class BlockManager( val idFromMaster = master.registerBlockManager( id, - maxOffHeapMemory, + maxOnHeapMemory, maxOffHeapMemory, slaveEndpoint) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 9b2f9543a23a5..c202cee34ec84 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -210,14 +210,14 @@ class StorageStatus( def offHeapMemUsed: Long = _nonRddStorageInfo._2 + offHeapCacheSize /** Return the memory used by on-heap caching RDDs */ - def onHeapCacheSize: Long = { - _rddStorageInfo.filter(!_._2._3.useOffHeap).map(_._2._1).sum - } + def onHeapCacheSize: Long = _rddStorageInfo.collect { + case (_, (memoryUsed, _, storageLevel)) if !storageLevel.useOffHeap => memoryUsed + }.sum /** Return the memory used by off-heap caching RDDs */ - def offHeapCacheSize: Long = { - _rddStorageInfo.filter(_._2._3.useOffHeap).map(_._2._1).sum - } + def offHeapCacheSize: Long = _rddStorageInfo.collect { + case (_, (memoryUsed, _, storageLevel)) if storageLevel.useOffHeap => memoryUsed + }.sum /** Return the disk space used by this block manager. */ def diskUsed: Long = _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 9f55f94675f35..887fd17b35562 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -55,6 +55,34 @@ private[ui] class ExecutorsPage( val content =
{ +
+ + + Show Additional Metrics + + +
++
++ ++ ++ @@ -67,6 +95,11 @@ private[ui] class ExecutorsPage( } private[spark] object ExecutorsPage { + private val ON_HEAP_MEMORY_TOOLTIP = "Memory used / total available memory for on heap " + + "storage of data like RDD partitions cached in memory." + private val OFF_HEAP_MEMORY_TOOLTIP = "Memory used / total available memory for off heap " + + "storage of data like RDD partitions cached in memory." + /** Represent an executor's info as a map given a storage status index */ def getExecInfo( listener: ExecutorsListener, diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 227e940c9c50c..b07ff4d302840 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -147,7 +147,8 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { /** Header fields for the worker table */ private def workerHeader = Seq( "Host", - "Memory Usage", + "On Heap Memory Usage", + "Off Heap Memory Usage", "Disk Usage") /** Render an HTML row representing a worker */ @@ -155,8 +156,12 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { {worker.address} - {Utils.bytesToString(worker.memoryUsed)} - ({Utils.bytesToString(worker.memoryRemaining)} Remaining) + {Utils.bytesToString(worker.onHeapMemoryUsed)} + ({Utils.bytesToString(worker.onHeapMemoryRemaining)} Remaining) + + + {Utils.bytesToString(worker.offHeapMemoryUsed)} + ({Utils.bytesToString(worker.offHeapMemoryRemaining)} Remaining) {Utils.bytesToString(worker.diskUsed)} diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json index 5914a1c2c4b6d..f03f50fb7e9ba 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json @@ -21,7 +21,11 @@ "executorLogs" : { "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" - } + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 0 }, { "id" : "driver", "hostPort" : "172.22.0.167:51475", @@ -42,7 +46,11 @@ "totalShuffleWrite" : 0, "isBlacklisted" : true, "maxMemory" : 384093388, - "executorLogs" : { } + "executorLogs" : { }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 0 }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -66,7 +74,11 @@ "executorLogs" : { "stdout" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout", "stderr" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr" - } + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 0 }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -90,7 +102,11 @@ "executorLogs" : { "stdout" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout", "stderr" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr" - } + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 0 }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -114,5 +130,9 @@ "executorLogs" : { "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" - } + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json index 92e249c851116..c15d4e1707061 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json @@ -21,7 +21,11 @@ "executorLogs" : { "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" - } + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 0 }, { "id" : "driver", "hostPort" : "172.22.0.111:64527", @@ -42,7 +46,11 @@ "totalShuffleWrite" : 0, "isBlacklisted" : false, "maxMemory" : 384093388, - "executorLogs" : { } + "executorLogs" : { }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 0 }, { "id" : "1", "hostPort" : "172.22.0.111:64541", @@ -66,7 +74,11 @@ "executorLogs" : { "stdout" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stdout", "stderr" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr" - } + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 0 }, { "id" : "0", "hostPort" : "172.22.0.111:64540", @@ -90,7 +102,11 @@ "executorLogs" : { "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" - } + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 0 }, { "id" : "3", "hostPort" : "172.22.0.111:64543", @@ -114,5 +130,9 @@ "executorLogs" : { "stdout" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout", "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" - } + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 0 } ] From 95574d5b355b0cddee8aec89bee19668ecaa1bd3 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 22 Mar 2017 11:14:56 +0800 Subject: [PATCH 06/11] Minor style fix Change-Id: I469532f3fb4ce9fda243cf2a812ad3e8b3b2aba5 --- .../resources/org/apache/spark/ui/static/executorspage.js | 7 ------- .../org/apache/spark/storage/BlockManagerMaster.scala | 2 +- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index b8089a502219c..fce0b48c13c07 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -401,7 +401,6 @@ $(document).ready(function () { var maxMemory = row.maxOnHeapMemory + row.maxOffHeapMemory; return (formatBytes(memoryUsed, type) + ' / ' + formatBytes(maxMemory, type)); - } }, { @@ -411,7 +410,6 @@ $(document).ready(function () { else return (formatBytes(row.onHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOnHeapMemory, type)); - }, "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { $(nTd).addClass('on_heap_memory') @@ -424,7 +422,6 @@ $(document).ready(function () { else return (formatBytes(row.offHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOffHeapMemory, type)); - }, "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { $(nTd).addClass('off_heap_memory') @@ -506,7 +503,6 @@ $(document).ready(function () { var maxMemory = row.allOnHeapMaxMemory + row.allOffHeapMaxMemory; return (formatBytes(memoryUsed, type) + ' / ' + formatBytes(maxMemory, type)); - } }, { @@ -516,7 +512,6 @@ $(document).ready(function () { else return (formatBytes(row.allOnHeapMemoryUsed, type) + ' / ' + formatBytes(row.allOnHeapMaxMemory, type)); - }, "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { $(nTd).addClass('on_heap_memory') @@ -529,12 +524,10 @@ $(document).ready(function () { else return (formatBytes(row.allOffHeapMemoryUsed, type) + ' / ' + formatBytes(row.allOffHeapMaxMemory, type)); - }, "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { $(nTd).addClass('off_heap_memory') } - }, {data: 'allDiskUsed', render: formatBytes}, {data: 'allTotalCores'}, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index d83e02f8d64f2..ea5d8423a588c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -63,7 +63,7 @@ class BlockManagerMaster( logInfo(s"Registering BlockManager $blockManagerId") val updatedId = driverEndpoint.askSync[BlockManagerId]( RegisterBlockManager(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) - logInfo(s"Registered BlockManager $blockManagerId") + logInfo(s"Registered BlockManager $updatedId") updatedId } From a74708943e0480ff6498c43f4b2a3421abc60266 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 22 Mar 2017 11:17:28 +0800 Subject: [PATCH 07/11] Add Mima exclusion Change-Id: Ibdec765efafa12045f56f1f8c0b7037fcc6f3224 --- project/MimaExcludes.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index e07a0a82c3560..feae76a087dec 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -107,7 +107,8 @@ object MimaExcludes { ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded$"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerBlockManagerAdded.apply"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.StorageStatus.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.this") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.RDDDataDistribution.this") ) // Exclude rules for 2.1.x From 80ed55f4551e211891d5e295f68317cf1d7c267f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 23 Mar 2017 14:22:11 +0800 Subject: [PATCH 08/11] Address the comments Change-Id: I3bed9bd91458519e6f2ea4844d685e04311cc7ee --- .../apache/spark/ui/static/executorspage.js | 6 +-- .../spark/storage/BlockManagerSource.scala | 4 +- .../apache/spark/storage/StorageUtils.scala | 44 +++++++++---------- 3 files changed, 27 insertions(+), 27 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index fce0b48c13c07..569072224c59a 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -395,7 +395,7 @@ $(document).ready(function () { { data: function (row, type) { if (type !== 'display') - return row.maxOnHeapMemory + row.maxOffHeapMemory; + return row.onHeapMemoryUsed + row.offHeapMemoryUsed; else var memoryUsed = row.onHeapMemoryUsed + row.offHeapMemoryUsed; var maxMemory = row.maxOnHeapMemory + row.maxOffHeapMemory; @@ -406,7 +406,7 @@ $(document).ready(function () { { data: function (row, type) { if (type !== 'display') - return row.maxOnHeapMemory; + return row.onHeapMemoryUsed; else return (formatBytes(row.onHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOnHeapMemory, type)); @@ -418,7 +418,7 @@ $(document).ready(function () { { data: function (row, type) { if (type !== 'display') - return row.maxOffHeapMemory; + return row.offHeapMemoryUsed; else return (formatBytes(row.offHeapMemoryUsed, type) + ' / ' + formatBytes(row.maxOffHeapMemory, type)); diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index c19b0f44d080a..741edfb72caa3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -26,9 +26,9 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) override val metricRegistry = new MetricRegistry() override val sourceName = "BlockManager" - private def registerGauge(name: String, f: BlockManagerMaster => Long): Unit = { + private def registerGauge(name: String, func: BlockManagerMaster => Long): Unit = { metricRegistry.register(name, new Gauge[Long] { - override def getValue: Long = f(blockManager.master) / 1024 / 1024 + override def getValue: Long = func(blockManager.master) / 1024 / 1024 }) } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index c202cee34ec84..c5cca2955392f 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -50,22 +50,22 @@ class StorageStatus( private val _nonRddBlocks = new mutable.HashMap[BlockId, BlockStatus] /** - * Storage information of the blocks that entails memory, disk, and off-heap memory usage. + * Storage information of the blocks that entails on-heap memory, off-heap memory and disk usage. * * As with the block maps, we store the storage information separately for RDD blocks and * non-RDD blocks for the same reason. In particular, RDD storage information is stored - * in a map indexed by the RDD ID to the following 3-tuple: + * in a map indexed by the RDD ID to the following 3-tuple case class: * * (memory size, disk size, storage level) * * We assume that all the blocks that belong to the same RDD have the same storage level. - * This field is not relevant to non-RDD blocks, however, so the storage information for - * non-RDD blocks contains only the first 3 fields (in the same order). */ - private val _rddStorageInfo = new mutable.HashMap[Int, (Long, Long, StorageLevel)] + case class RddStorageInfo(memoryUsage: Long, diskUsage: Long, level: StorageLevel) + private val _rddStorageInfo = new mutable.HashMap[Int, RddStorageInfo] // On-heap memory, off-heap memory and disk usage of non rdd storage - private var _nonRddStorageInfo: (Long, Long, Long) = (0L, 0L, 0L) + case class NonRddStorageInfo(var onHeapUsage: Long, var offHeapUsage: Long, var diskUsage: Long) + private val _nonRddStorageInfo = NonRddStorageInfo(0L, 0L, 0L) /** Create a storage status with an initial set of blocks, leaving the source unmodified. */ def this( @@ -204,32 +204,32 @@ class StorageStatus( def offHeapMemRemaining: Long = maxOffHeapMem - offHeapMemUsed /** Return the on-heap memory used by this block manager. */ - def onHeapMemUsed: Long = _nonRddStorageInfo._1 + onHeapCacheSize + def onHeapMemUsed: Long = _nonRddStorageInfo.onHeapUsage + onHeapCacheSize /** Return the off-heap memory used by this block manager. */ - def offHeapMemUsed: Long = _nonRddStorageInfo._2 + offHeapCacheSize + def offHeapMemUsed: Long = _nonRddStorageInfo.offHeapUsage + offHeapCacheSize /** Return the memory used by on-heap caching RDDs */ def onHeapCacheSize: Long = _rddStorageInfo.collect { - case (_, (memoryUsed, _, storageLevel)) if !storageLevel.useOffHeap => memoryUsed + case (_, storageInfo) if !storageInfo.level.useOffHeap => storageInfo.memoryUsage }.sum /** Return the memory used by off-heap caching RDDs */ def offHeapCacheSize: Long = _rddStorageInfo.collect { - case (_, (memoryUsed, _, storageLevel)) if storageLevel.useOffHeap => memoryUsed + case (_, storageInfo) if storageInfo.level.useOffHeap => storageInfo.memoryUsage }.sum /** Return the disk space used by this block manager. */ - def diskUsed: Long = _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum + def diskUsed: Long = _nonRddStorageInfo.diskUsage + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum /** Return the memory used by the given RDD in this block manager in O(1) time. */ - def memUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._1).getOrElse(0L) + def memUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_.memoryUsage).getOrElse(0L) /** Return the disk space used by the given RDD in this block manager in O(1) time. */ - def diskUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._2).getOrElse(0L) + def diskUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_.diskUsage).getOrElse(0L) /** Return the storage level, if any, used by the given RDD in this block manager. */ - def rddStorageLevel(rddId: Int): Option[StorageLevel] = _rddStorageInfo.get(rddId).map(_._3) + def rddStorageLevel(rddId: Int): Option[StorageLevel] = _rddStorageInfo.get(rddId).map(_.level) /** * Update the relevant storage info, taking into account any existing status for this block. @@ -244,12 +244,12 @@ class StorageStatus( val (oldMem, oldDisk) = blockId match { case RDDBlockId(rddId, _) => _rddStorageInfo.get(rddId) - .map { case (mem, disk, _) => (mem, disk) } + .map { case RddStorageInfo(mem, disk, _) => (mem, disk) } .getOrElse((0L, 0L)) case _ if !level.useOffHeap => - (_nonRddStorageInfo._1, _nonRddStorageInfo._3) + (_nonRddStorageInfo.onHeapUsage, _nonRddStorageInfo.diskUsage) case _ if level.useOffHeap => - (_nonRddStorageInfo._2, _nonRddStorageInfo._3) + (_nonRddStorageInfo.offHeapUsage, _nonRddStorageInfo.diskUsage) } val newMem = math.max(oldMem + changeInMem, 0L) val newDisk = math.max(oldDisk + changeInDisk, 0L) @@ -261,17 +261,17 @@ class StorageStatus( if (newMem + newDisk == 0) { _rddStorageInfo.remove(rddId) } else { - _rddStorageInfo(rddId) = (newMem, newDisk, level) + _rddStorageInfo(rddId) = RddStorageInfo(newMem, newDisk, level) } case _ => - _nonRddStorageInfo = if (!level.useOffHeap) { - (newMem, _nonRddStorageInfo._2, newDisk) + if (!level.useOffHeap) { + _nonRddStorageInfo.onHeapUsage = newMem } else { - (_nonRddStorageInfo._1, newMem, newDisk) + _nonRddStorageInfo.offHeapUsage = newMem } + _nonRddStorageInfo.diskUsage = newDisk } } - } /** Helper methods for storage-related objects. */ From 60383b9fd7de6d8a089f73b1f4f1e8e132f46a51 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 27 Mar 2017 16:34:50 +0800 Subject: [PATCH 09/11] Adress some style issues and change event log to reflect off memory usage Change-Id: I2803e2efba7607de7e9a9f1c04556250ca698625 --- .../spark/storage/StorageStatusListener.scala | 2 + .../apache/spark/storage/StorageUtils.scala | 17 +-- .../executor_memory_usage_expectation.json | 139 ++++++++++++++++++ ...xecutor_node_blacklisting_expectation.json | 21 +-- .../spark-events/app-20161116163331-0000 | 10 +- .../deploy/history/HistoryServerSuite.scala | 3 +- 6 files changed, 162 insertions(+), 30 deletions(-) create mode 100644 core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 20d56fa125f91..3cbeeac8059db 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -74,6 +74,8 @@ class StorageStatusListener(conf: SparkConf) extends SparkListener { synchronized { val blockManagerId = blockManagerAdded.blockManagerId val executorId = blockManagerId.executorId + // This two fields are compatible with old event logs, in which there only has max on heap + // memory in the event log. So maxOnHeapMem will use maxMem, maxOffHeapMem will set to 0. val maxOnHeapMem = blockManagerAdded.maxOnHeapMem.getOrElse(blockManagerAdded.maxMem) val maxOffHeapMem = blockManagerAdded.maxOffHeapMem.getOrElse(0L) val storageStatus = new StorageStatus(blockManagerId, maxOnHeapMem, maxOffHeapMem) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index c5cca2955392f..cc78cde2dfd04 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -49,22 +49,11 @@ class StorageStatus( private val _rddBlocks = new mutable.HashMap[Int, mutable.Map[BlockId, BlockStatus]] private val _nonRddBlocks = new mutable.HashMap[BlockId, BlockStatus] - /** - * Storage information of the blocks that entails on-heap memory, off-heap memory and disk usage. - * - * As with the block maps, we store the storage information separately for RDD blocks and - * non-RDD blocks for the same reason. In particular, RDD storage information is stored - * in a map indexed by the RDD ID to the following 3-tuple case class: - * - * (memory size, disk size, storage level) - * - * We assume that all the blocks that belong to the same RDD have the same storage level. - */ - case class RddStorageInfo(memoryUsage: Long, diskUsage: Long, level: StorageLevel) + private case class RddStorageInfo(memoryUsage: Long, diskUsage: Long, level: StorageLevel) private val _rddStorageInfo = new mutable.HashMap[Int, RddStorageInfo] - // On-heap memory, off-heap memory and disk usage of non rdd storage - case class NonRddStorageInfo(var onHeapUsage: Long, var offHeapUsage: Long, var diskUsage: Long) + private case class NonRddStorageInfo(var onHeapUsage: Long, var offHeapUsage: Long, + var diskUsage: Long) private val _nonRddStorageInfo = NonRddStorageInfo(0L, 0L, 0L) /** Create a storage status with an initial set of blocks, leaving the source unmodified. */ diff --git a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json new file mode 100644 index 0000000000000..e732af2663503 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json @@ -0,0 +1,139 @@ +[ { + "id" : "2", + "hostPort" : "172.22.0.167:51487", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2537, + "totalGCTime" : 88, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 524288000 +}, { + "id" : "driver", + "hostPort" : "172.22.0.167:51475", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 524288000 +}, { + "id" : "1", + "hostPort" : "172.22.0.167:51490", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 4, + "totalTasks" : 4, + "totalDuration" : 3152, + "totalGCTime" : 68, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout", + "stderr" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr" + }, + + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 524288000 +}, { + "id" : "0", + "hostPort" : "172.22.0.167:51491", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2551, + "totalGCTime" : 116, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr" + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 524288000 +}, { + "id" : "3", + "hostPort" : "172.22.0.167:51485", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 12, + "totalTasks" : 12, + "totalDuration" : 2453, + "totalGCTime" : 72, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" + }, + "onHeapMemoryUsed" : 0, + "offHeapMemoryUsed" : 0, + "maxOnHeapMemory" : 384093388, + "maxOffHeapMemory" : 524288000 +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json index f03f50fb7e9ba..e732af2663503 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json @@ -17,7 +17,7 @@ "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : true, - "maxMemory" : 384093388, + "maxMemory" : 908381388, "executorLogs" : { "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" @@ -25,7 +25,7 @@ "onHeapMemoryUsed" : 0, "offHeapMemoryUsed" : 0, "maxOnHeapMemory" : 384093388, - "maxOffHeapMemory" : 0 + "maxOffHeapMemory" : 524288000 }, { "id" : "driver", "hostPort" : "172.22.0.167:51475", @@ -45,12 +45,12 @@ "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : true, - "maxMemory" : 384093388, + "maxMemory" : 908381388, "executorLogs" : { }, "onHeapMemoryUsed" : 0, "offHeapMemoryUsed" : 0, "maxOnHeapMemory" : 384093388, - "maxOffHeapMemory" : 0 + "maxOffHeapMemory" : 524288000 }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -70,15 +70,16 @@ "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : true, - "maxMemory" : 384093388, + "maxMemory" : 908381388, "executorLogs" : { "stdout" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout", "stderr" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr" }, + "onHeapMemoryUsed" : 0, "offHeapMemoryUsed" : 0, "maxOnHeapMemory" : 384093388, - "maxOffHeapMemory" : 0 + "maxOffHeapMemory" : 524288000 }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -98,7 +99,7 @@ "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : true, - "maxMemory" : 384093388, + "maxMemory" : 908381388, "executorLogs" : { "stdout" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout", "stderr" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr" @@ -106,7 +107,7 @@ "onHeapMemoryUsed" : 0, "offHeapMemoryUsed" : 0, "maxOnHeapMemory" : 384093388, - "maxOffHeapMemory" : 0 + "maxOffHeapMemory" : 524288000 }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -126,7 +127,7 @@ "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : true, - "maxMemory" : 384093388, + "maxMemory" : 908381388, "executorLogs" : { "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" @@ -134,5 +135,5 @@ "onHeapMemoryUsed" : 0, "offHeapMemoryUsed" : 0, "maxOnHeapMemory" : 384093388, - "maxOffHeapMemory" : 0 + "maxOffHeapMemory" : 524288000 } ] diff --git a/core/src/test/resources/spark-events/app-20161116163331-0000 b/core/src/test/resources/spark-events/app-20161116163331-0000 index 7566c9fc0a20b..57cfc5b973129 100755 --- a/core/src/test/resources/spark-events/app-20161116163331-0000 +++ b/core/src/test/resources/spark-events/app-20161116163331-0000 @@ -1,15 +1,15 @@ {"Event":"SparkListenerLogStart","Spark Version":"2.1.0-SNAPSHOT"} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"172.22.0.167","Port":51475},"Maximum Memory":384093388,"Timestamp":1479335611477} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"172.22.0.167","Port":51475},"Maximum Memory":908381388,"Timestamp":1479335611477,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.167","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"51459","spark.repl.class.uri":"spark://172.22.0.167:51459/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"1000000","spark.app.id":"app-20161116163331-0000","spark.task.maxFailures":"4"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} {"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20161116163331-0000","Timestamp":1479335609916,"User":"jose"} {"Event":"SparkListenerExecutorAdded","Timestamp":1479335615320,"Executor ID":"3","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout","stderr":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"172.22.0.167","Port":51485},"Maximum Memory":384093388,"Timestamp":1479335615387} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"172.22.0.167","Port":51485},"Maximum Memory":908381388,"Timestamp":1479335615387,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} {"Event":"SparkListenerExecutorAdded","Timestamp":1479335615393,"Executor ID":"2","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout","stderr":"http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr"}}} {"Event":"SparkListenerExecutorAdded","Timestamp":1479335615443,"Executor ID":"1","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout","stderr":"http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"172.22.0.167","Port":51487},"Maximum Memory":384093388,"Timestamp":1479335615448} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"172.22.0.167","Port":51487},"Maximum Memory":908381388,"Timestamp":1479335615448,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} {"Event":"SparkListenerExecutorAdded","Timestamp":1479335615462,"Executor ID":"0","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout","stderr":"http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"172.22.0.167","Port":51490},"Maximum Memory":384093388,"Timestamp":1479335615496} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"172.22.0.167","Port":51491},"Maximum Memory":384093388,"Timestamp":1479335615515} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"172.22.0.167","Port":51490},"Maximum Memory":908381388,"Timestamp":1479335615496,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"172.22.0.167","Port":51491},"Maximum Memory":908381388,"Timestamp":1479335615515,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} {"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1479335616467,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]}],"Stage IDs":[0],"Properties":{}} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]},"Properties":{}} {"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479335616657,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index dcf83cb530a91..764156c3edc41 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -153,7 +153,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "rdd list storage json" -> "applications/local-1422981780767/storage/rdd", "executor node blacklisting" -> "applications/app-20161116163331-0000/executors", - "executor node blacklisting unblacklisting" -> "applications/app-20161115172038-0000/executors" + "executor node blacklisting unblacklisting" -> "applications/app-20161115172038-0000/executors", + "executor memory usage" -> "applications/app-20161116163331-0000/executors" // Todo: enable this test when logging the even of onBlockUpdated. See: SPARK-13845 // "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" ) From b30e7d0c2e950179ef5801a697215ec9afd88226 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Sat, 1 Apr 2017 18:31:20 +0800 Subject: [PATCH 10/11] Change the way to be compatible with old event format Change-Id: I06acc1c194f90880b28e1992618419ff5bd2eb46 --- .../apache/spark/ui/static/executorspage.js | 43 ++++++++++++++----- .../spark/status/api/v1/AllRDDResource.scala | 6 ++- .../org/apache/spark/status/api/v1/api.scala | 8 ++-- .../storage/BlockManagerMasterEndpoint.scala | 3 +- .../spark/storage/BlockManagerSource.scala | 8 ++-- .../spark/storage/StorageStatusListener.scala | 9 ++-- .../apache/spark/storage/StorageUtils.scala | 22 ++++++---- .../apache/spark/ui/exec/ExecutorsPage.scala | 5 ++- .../executor_list_json_expectation.json | 6 +-- ...acklisting_unblacklisting_expectation.json | 30 +++---------- .../apache/spark/storage/StorageSuite.scala | 38 ++++++++++++---- .../org/apache/spark/ui/UISeleniumSuite.scala | 28 +++++++++++- 12 files changed, 128 insertions(+), 78 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 569072224c59a..930a0698928d1 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -188,6 +188,8 @@ $(document).ready(function () { var summary = []; var allExecCnt = 0; var allRDDBlocks = 0; + var allMemoryUsed = 0; + var allMaxMemory = 0; var allOnHeapMemoryUsed = 0; var allOnHeapMaxMemory = 0; var allOffHeapMemoryUsed = 0; @@ -208,6 +210,8 @@ $(document).ready(function () { var activeExecCnt = 0; var activeRDDBlocks = 0; + var activeMemoryUsed = 0; + var activeMaxMemory = 0; var activeOnHeapMemoryUsed = 0; var activeOnHeapMaxMemory = 0; var activeOffHeapMemoryUsed = 0; @@ -228,6 +232,8 @@ $(document).ready(function () { var deadExecCnt = 0; var deadRDDBlocks = 0; + var deadMemoryUsed = 0; + var deadMaxMemory = 0; var deadOnHeapMemoryUsed = 0; var deadOnHeapMaxMemory = 0; var deadOffHeapMemoryUsed = 0; @@ -246,9 +252,18 @@ $(document).ready(function () { var deadTotalShuffleWrite = 0; var deadTotalBlacklisted = 0; + response.forEach(function (exec) { + exec.onHeapMemoryUsed = exec.hasOwnProperty('onHeapMemoryUsed') ? exec.onHeapMemoryUsed : 0; + exec.maxOnHeapMemory = exec.hasOwnProperty('maxOnHeapMemory') ? exec.maxOnHeapMemory : 0; + exec.offHeapMemoryUsed = exec.hasOwnProperty('offHeapMemoryUsed') ? exec.offHeapMemoryUsed : 0; + exec.maxOffHeapMemory = exec.hasOwnProperty('maxOffHeapMemory') ? exec.maxOffHeapMemory : 0; + }); + response.forEach(function (exec) { allExecCnt += 1; allRDDBlocks += exec.rddBlocks; + allMemoryUsed += exec.memoryUsed; + allMaxMemory += exec.maxMemory; allOnHeapMemoryUsed += exec.onHeapMemoryUsed; allOnHeapMaxMemory += exec.maxOnHeapMemory; allOffHeapMemoryUsed += exec.offHeapMemoryUsed; @@ -269,6 +284,8 @@ $(document).ready(function () { if (exec.isActive) { activeExecCnt += 1; activeRDDBlocks += exec.rddBlocks; + activeMemoryUsed += exec.memoryUsed; + activeMaxMemory += exec.maxMemory; activeOnHeapMemoryUsed += exec.onHeapMemoryUsed; activeOnHeapMaxMemory += exec.maxOnHeapMemory; activeOffHeapMemoryUsed += exec.offHeapMemoryUsed; @@ -291,6 +308,10 @@ $(document).ready(function () { deadRDDBlocks += exec.rddBlocks; deadMemoryUsed += exec.memoryUsed; deadMaxMemory += exec.maxMemory; + deadOnHeapMemoryUsed += exec.onHeapMemoryUsed; + deadOnHeapMaxMemory += exec.maxOnHeapMemory; + deadOffHeapMemoryUsed += exec.offHeapMemoryUsed; + deadOffHeapMaxMemory += exec.maxOffHeapMemory; deadDiskUsed += exec.diskUsed; deadTotalCores += exec.totalCores; deadMaxTasks += exec.maxTasks; @@ -310,6 +331,8 @@ $(document).ready(function () { var totalSummary = { "execCnt": ( "Total(" + allExecCnt + ")"), "allRDDBlocks": allRDDBlocks, + "allMemoryUsed": allMemoryUsed, + "allMaxMemory": allMaxMemory, "allOnHeapMemoryUsed": allOnHeapMemoryUsed, "allOnHeapMaxMemory": allOnHeapMaxMemory, "allOffHeapMemoryUsed": allOffHeapMemoryUsed, @@ -331,6 +354,8 @@ $(document).ready(function () { var activeSummary = { "execCnt": ( "Active(" + activeExecCnt + ")"), "allRDDBlocks": activeRDDBlocks, + "allMemoryUsed": activeMemoryUsed, + "allMaxMemory": activeMaxMemory, "allOnHeapMemoryUsed": activeOnHeapMemoryUsed, "allOnHeapMaxMemory": activeOnHeapMaxMemory, "allOffHeapMemoryUsed": activeOffHeapMemoryUsed, @@ -352,6 +377,8 @@ $(document).ready(function () { var deadSummary = { "execCnt": ( "Dead(" + deadExecCnt + ")" ), "allRDDBlocks": deadRDDBlocks, + "allMemoryUsed": deadMemoryUsed, + "allMaxMemory": deadMaxMemory, "allOnHeapMemoryUsed": deadOnHeapMemoryUsed, "allOnHeapMaxMemory": deadOnHeapMaxMemory, "allOffHeapMemoryUsed": deadOffHeapMemoryUsed, @@ -395,12 +422,10 @@ $(document).ready(function () { { data: function (row, type) { if (type !== 'display') - return row.onHeapMemoryUsed + row.offHeapMemoryUsed; + return row.memoryUsed; else - var memoryUsed = row.onHeapMemoryUsed + row.offHeapMemoryUsed; - var maxMemory = row.maxOnHeapMemory + row.maxOffHeapMemory; - return (formatBytes(memoryUsed, type) + ' / ' + - formatBytes(maxMemory, type)); + return (formatBytes(row.memoryUsed, type) + ' / ' + + formatBytes(row.maxMemory, type)); } }, { @@ -497,12 +522,10 @@ $(document).ready(function () { { data: function (row, type) { if (type !== 'display') - return row.allOnHeapMemoryUsed + row.allOffHeapMemoryUsed; + return row.allMemoryUsed else - var memoryUsed = row.allOnHeapMemoryUsed + row.allOffHeapMemoryUsed; - var maxMemory = row.allOnHeapMaxMemory + row.allOffHeapMaxMemory; - return (formatBytes(memoryUsed, type) + ' / ' + - formatBytes(maxMemory, type)); + return (formatBytes(row.allMemoryUsed, type) + ' / ' + + formatBytes(row.allMaxMemory, type)); } }, { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala index e43252c53cc82..b352f84eb39b0 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala @@ -75,8 +75,10 @@ private[spark] object AllRDDResource { if (!rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L, offHeapMemoryUsed = if (rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L, - onHeapMemoryRemaining = status.onHeapMemRemaining, - offHeapMemoryRemaining = status.offHeapMemRemaining + // Get RDDStorageInfo REST API will return RDD information only in the Live UI, so + // we assume this two fields always exist. + onHeapMemoryRemaining = status.onHeapMemRemaining.get, + offHeapMemoryRemaining = status.offHeapMemRemaining.get ) } ) } else { None diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index f4d58e38e360e..9b63c89ad77b3 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -76,10 +76,10 @@ class ExecutorSummary private[spark]( val isBlacklisted: Boolean, val maxMemory: Long, val executorLogs: Map[String, String], - val onHeapMemoryUsed: Long, - val offHeapMemoryUsed: Long, - val maxOnHeapMemory: Long, - val maxOffHeapMemory: Long) + val onHeapMemoryUsed: Option[Long], + val offHeapMemoryUsed: Option[Long], + val maxOnHeapMemory: Option[Long], + val maxOffHeapMemory: Option[Long]) class JobData private[spark]( val jobId: Int, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 9c2e8e10ce2d3..467c3e0e6b51f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -276,7 +276,8 @@ class BlockManagerMasterEndpoint( private def storageStatus: Array[StorageStatus] = { blockManagerInfo.map { case (blockManagerId, info) => - new StorageStatus(blockManagerId, info.maxOnHeapMem, info.maxOffHeapMem, info.blocks.asScala) + new StorageStatus(blockManagerId, info.maxMem, Some(info.maxOnHeapMem), + Some(info.maxOffHeapMem), info.blocks.asScala) }.toArray } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 741edfb72caa3..b6cb041138e9d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -36,19 +36,19 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) _.getStorageStatus.map(_.maxMem).sum) registerGauge(MetricRegistry.name("memory", "maxOnHeapMem_MB"), - _.getStorageStatus.map(_.maxOnHeapMem).sum) + _.getStorageStatus.map(_.maxOnHeapMem.getOrElse(0L)).sum) registerGauge(MetricRegistry.name("memory", "maxOffHeapMem_MB"), - _.getStorageStatus.map(_.maxOffHeapMem).sum) + _.getStorageStatus.map(_.maxOffHeapMem.getOrElse(0L)).sum) registerGauge(MetricRegistry.name("memory", "remainingMem_MB"), _.getStorageStatus.map(_.memRemaining).sum) registerGauge(MetricRegistry.name("memory", "remainingOnHeapMem_MB"), - _.getStorageStatus.map(_.onHeapMemRemaining).sum) + _.getStorageStatus.map(_.onHeapMemRemaining.getOrElse(0L)).sum) registerGauge(MetricRegistry.name("memory", "remainingOffHeapMem_MB"), - _.getStorageStatus.map(_.offHeapMemRemaining).sum) + _.getStorageStatus.map(_.offHeapMemRemaining.getOrElse(0L)).sum) registerGauge(MetricRegistry.name("memory", "memUsed_MB"), _.getStorageStatus.map(_.memUsed).sum) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 3cbeeac8059db..b5faed81350db 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -41,7 +41,7 @@ class StorageStatusListener(conf: SparkConf) extends SparkListener { } def deadStorageStatusList: Seq[StorageStatus] = synchronized { - deadExecutorStorageStatus.toSeq + deadExecutorStorageStatus } /** Update storage status list to reflect updated block statuses */ @@ -74,11 +74,8 @@ class StorageStatusListener(conf: SparkConf) extends SparkListener { synchronized { val blockManagerId = blockManagerAdded.blockManagerId val executorId = blockManagerId.executorId - // This two fields are compatible with old event logs, in which there only has max on heap - // memory in the event log. So maxOnHeapMem will use maxMem, maxOffHeapMem will set to 0. - val maxOnHeapMem = blockManagerAdded.maxOnHeapMem.getOrElse(blockManagerAdded.maxMem) - val maxOffHeapMem = blockManagerAdded.maxOffHeapMem.getOrElse(0L) - val storageStatus = new StorageStatus(blockManagerId, maxOnHeapMem, maxOffHeapMem) + val storageStatus = new StorageStatus(blockManagerId, blockManagerAdded.maxMem, + blockManagerAdded.maxOnHeapMem, blockManagerAdded.maxOffHeapMem) executorIdToStorageStatus(executorId) = storageStatus // Try to remove the dead storage status if same executor register the block manager twice. diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index cc78cde2dfd04..840966bed7d0e 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -37,8 +37,11 @@ import org.apache.spark.internal.Logging @DeveloperApi class StorageStatus( val blockManagerId: BlockManagerId, - val maxOnHeapMem: Long, - val maxOffHeapMem: Long) { + // Explicitly adding this maxMemory field to handle maxOnHeapMem and maxOffHeapMem not + // existing issue, this happened when trying to replay old event log. + val maxMemory: Long, + val maxOnHeapMem: Option[Long], + val maxOffHeapMem: Option[Long]) { /** * Internal representation of the blocks stored in this block manager. @@ -59,10 +62,11 @@ class StorageStatus( /** Create a storage status with an initial set of blocks, leaving the source unmodified. */ def this( bmid: BlockManagerId, - maxOnHeapMem: Long, - maxOffHeapMem: Long, + maxMemory: Long, + maxOnHeapMem: Option[Long], + maxOffHeapMem: Option[Long], initialBlocks: Map[BlockId, BlockStatus]) { - this(bmid, maxOnHeapMem, maxOffHeapMem) + this(bmid, maxMemory, maxOnHeapMem, maxOffHeapMem) initialBlocks.foreach { case (bid, bstatus) => addBlock(bid, bstatus) } } @@ -175,10 +179,10 @@ class StorageStatus( def numRddBlocksById(rddId: Int): Int = _rddBlocks.get(rddId).map(_.size).getOrElse(0) /** Return the max memory can be used by this block manager. */ - def maxMem: Long = maxOnHeapMem + maxOffHeapMem + def maxMem: Long = maxMemory /** Return the memory remaining in this block manager. */ - def memRemaining: Long = onHeapMemRemaining + offHeapMemRemaining + def memRemaining: Long = maxMem - memUsed /** Return the memory used by caching RDDs */ def cacheSize: Long = onHeapCacheSize + offHeapCacheSize @@ -187,10 +191,10 @@ class StorageStatus( def memUsed: Long = onHeapMemUsed + offHeapMemUsed /** Return the on-heap memory remaining in this block manager. */ - def onHeapMemRemaining: Long = maxOnHeapMem - onHeapMemUsed + def onHeapMemRemaining: Option[Long] = maxOnHeapMem.map(_ - onHeapMemUsed) /** Return the off-heap memory remaining in this block manager. */ - def offHeapMemRemaining: Long = maxOffHeapMem - offHeapMemUsed + def offHeapMemRemaining: Option[Long] = maxOffHeapMem.map(_ - offHeapMemUsed) /** Return the on-heap memory used by this block manager. */ def onHeapMemUsed: Long = _nonRddStorageInfo.onHeapUsage + onHeapCacheSize diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 887fd17b35562..494b69c3412cc 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -115,8 +115,9 @@ private[spark] object ExecutorsPage { val rddBlocks = status.numBlocks val memUsed = status.memUsed val maxMem = status.maxMem - val onHeapMemUsed = status.onHeapMemUsed - val offHeapMemUsed = status.offHeapMemUsed + // Only maxOnHeapMem and maxOffHeapMem are defined these two fields are not None. + val onHeapMemUsed = status.maxOnHeapMem.map(_ => status.onHeapMemUsed) + val offHeapMemUsed = status.maxOffHeapMem.map(_ => status.offHeapMemUsed) val maxOnHeapMem = status.maxOnHeapMem val maxOffHeapMem = status.maxOffHeapMem val diskUsed = status.diskUsed diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index f379151f91e0b..6b9f29e1a230e 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -18,9 +18,5 @@ "totalShuffleWrite" : 13180, "isBlacklisted" : false, "maxMemory" : 278302556, - "executorLogs" : { }, - "onHeapMemoryUsed": 0, - "offHeapMemoryUsed": 0, - "maxOnHeapMemory": 278302556, - "maxOffHeapMemory": 0 + "executorLogs" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json index c15d4e1707061..92e249c851116 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json @@ -21,11 +21,7 @@ "executorLogs" : { "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" - }, - "onHeapMemoryUsed" : 0, - "offHeapMemoryUsed" : 0, - "maxOnHeapMemory" : 384093388, - "maxOffHeapMemory" : 0 + } }, { "id" : "driver", "hostPort" : "172.22.0.111:64527", @@ -46,11 +42,7 @@ "totalShuffleWrite" : 0, "isBlacklisted" : false, "maxMemory" : 384093388, - "executorLogs" : { }, - "onHeapMemoryUsed" : 0, - "offHeapMemoryUsed" : 0, - "maxOnHeapMemory" : 384093388, - "maxOffHeapMemory" : 0 + "executorLogs" : { } }, { "id" : "1", "hostPort" : "172.22.0.111:64541", @@ -74,11 +66,7 @@ "executorLogs" : { "stdout" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stdout", "stderr" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr" - }, - "onHeapMemoryUsed" : 0, - "offHeapMemoryUsed" : 0, - "maxOnHeapMemory" : 384093388, - "maxOffHeapMemory" : 0 + } }, { "id" : "0", "hostPort" : "172.22.0.111:64540", @@ -102,11 +90,7 @@ "executorLogs" : { "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" - }, - "onHeapMemoryUsed" : 0, - "offHeapMemoryUsed" : 0, - "maxOnHeapMemory" : 384093388, - "maxOffHeapMemory" : 0 + } }, { "id" : "3", "hostPort" : "172.22.0.111:64543", @@ -130,9 +114,5 @@ "executorLogs" : { "stdout" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout", "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" - }, - "onHeapMemoryUsed" : 0, - "offHeapMemoryUsed" : 0, - "maxOnHeapMemory" : 384093388, - "maxOffHeapMemory" : 0 + } } ] diff --git a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala index 0ea0cd5f5557d..5bdd81fab048b 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -27,7 +27,7 @@ class StorageSuite extends SparkFunSuite { // For testing add, update, and remove (for non-RDD blocks) private def storageStatus1: StorageStatus = { - val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, 0L) + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, Some(1000L), Some(0L)) assert(status.blocks.isEmpty) assert(status.rddBlocks.isEmpty) assert(status.memUsed === 0L) @@ -74,7 +74,7 @@ class StorageSuite extends SparkFunSuite { // For testing add, update, remove, get, and contains etc. for both RDD and non-RDD blocks private def storageStatus2: StorageStatus = { - val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, 0L) + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, Some(1000L), Some(0L)) assert(status.rddBlocks.isEmpty) status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L)) status.addBlock(TestBlockId("man"), BlockStatus(memAndDisk, 10L, 20L)) @@ -252,9 +252,9 @@ class StorageSuite extends SparkFunSuite { // For testing StorageUtils.updateRddInfo and StorageUtils.getRddBlockLocations private def stockStorageStatuses: Seq[StorageStatus] = { - val status1 = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, 0L) - val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2), 2000L, 0L) - val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3), 3000L, 0L) + val status1 = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, Some(1000L), Some(0L)) + val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2), 2000L, Some(2000L), Some(0L)) + val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3), 3000L, Some(3000L), Some(0L)) status1.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L)) status1.addBlock(RDDBlockId(0, 1), BlockStatus(memAndDisk, 1L, 2L)) status2.addBlock(RDDBlockId(0, 2), BlockStatus(memAndDisk, 1L, 2L)) @@ -336,7 +336,7 @@ class StorageSuite extends SparkFunSuite { // For testing add, update, remove, get, and contains etc. for both RDD and non-RDD onheap // and offheap blocks private def storageStatus3: StorageStatus = { - val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, 1000L) + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 2000L, Some(1000L), Some(1000L)) assert(status.rddBlocks.isEmpty) status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L)) status.addBlock(TestBlockId("man"), BlockStatus(offheap, 10L, 0L)) @@ -358,7 +358,7 @@ class StorageSuite extends SparkFunSuite { def actualOffHeapMemUsed: Long = status.blocks.values.filter(_.storageLevel.useOffHeap).map(_.memSize).sum - assert(status.maxMem === status.maxOnHeapMem + status.maxOffHeapMem) + assert(status.maxMem === status.maxOnHeapMem.get + status.maxOffHeapMem.get) assert(status.memUsed === actualMemUsed) assert(status.diskUsed === actualDiskUsed) @@ -366,8 +366,8 @@ class StorageSuite extends SparkFunSuite { assert(status.offHeapMemUsed === actualOffHeapMemUsed) assert(status.memRemaining === status.maxMem - actualMemUsed) - assert(status.onHeapMemRemaining === status.maxOnHeapMem - actualOnHeapMemUsed) - assert(status.offHeapMemRemaining === status.maxOffHeapMem - actualOffHeapMemUsed) + assert(status.onHeapMemRemaining.get === status.maxOnHeapMem.get - actualOnHeapMemUsed) + assert(status.offHeapMemRemaining.get === status.maxOffHeapMem.get - actualOffHeapMemUsed) status.addBlock(TestBlockId("wire"), BlockStatus(memAndDisk, 400L, 500L)) status.addBlock(RDDBlockId(25, 25), BlockStatus(memAndDisk, 40L, 50L)) @@ -389,4 +389,24 @@ class StorageSuite extends SparkFunSuite { assert(status.memUsed === actualMemUsed) assert(status.diskUsed === actualDiskUsed) } + + private def storageStatus4: StorageStatus = { + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 2000L, None, None) + status + } + test("old SparkListenerBlockManagerAdded event compatible") { + // This scenario will only be happened when replaying old event log. In this scenario there's + // no block add or remove event replayed, so only total amount of memory is valid. + val status = storageStatus4 + assert(status.maxMem === status.maxMemory) + + assert(status.memUsed === 0L) + assert(status.diskUsed === 0L) + assert(status.onHeapMemUsed === 0L) + assert(status.offHeapMemUsed === 0L) + + assert(status.memRemaining === status.maxMem) + assert(status.onHeapMemRemaining === None) + assert(status.offHeapMemRemaining === None) + } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 4228373036425..f65a40aefd964 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.LocalSparkContext._ import org.apache.spark.api.java.StorageLevels import org.apache.spark.deploy.history.HistoryServerSuite import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.status.api.v1.{JacksonMessageWriter, StageStatus} +import org.apache.spark.status.api.v1.{JacksonMessageWriter, RDDDataDistribution, StageStatus} private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler { @@ -103,6 +103,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B .set("spark.ui.enabled", "true") .set("spark.ui.port", "0") .set("spark.ui.killEnabled", killEnabled.toString) + .set("spark.memory.offHeap.size", "64m") val sc = new SparkContext(conf) assert(sc.ui.isDefined) sc @@ -151,6 +152,31 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val updatedRddJson = getJson(ui, "storage/rdd/0") (updatedRddJson \ "storageLevel").extract[String] should be ( StorageLevels.MEMORY_ONLY.description) + + val dataDistributions0 = + (updatedRddJson \ "dataDistribution").extract[Seq[RDDDataDistribution]] + dataDistributions0.length should be (1) + val distribution0 = dataDistributions0.head + distribution0.memoryUsed should be (distribution0.onHeapMemoryUsed) + distribution0.memoryRemaining should be ( + distribution0.onHeapMemoryRemaining + distribution0.offHeapMemoryRemaining) + distribution0.onHeapMemoryUsed should not be (0L) + distribution0.offHeapMemoryUsed should be (0L) + + rdd.unpersist() + rdd.persist(StorageLevels.OFF_HEAP).count() + val updatedStorageJson1 = getJson(ui, "storage/rdd") + updatedStorageJson1.children.length should be (1) + val updatedRddJson1 = getJson(ui, "storage/rdd/0") + val dataDistributions1 = + (updatedRddJson1 \ "dataDistribution").extract[Seq[RDDDataDistribution]] + dataDistributions1.length should be (1) + val distribution1 = dataDistributions1.head + distribution1.memoryUsed should be (distribution1.offHeapMemoryUsed) + distribution1.memoryRemaining should be ( + distribution1.onHeapMemoryRemaining + distribution1.offHeapMemoryRemaining) + distribution1.onHeapMemoryUsed should be (0L) + distribution1.offHeapMemoryUsed should not be (0L) } } From 643552c5113ea86bca67cca9c06fb8d8eb71f4af Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 5 Apr 2017 17:15:58 +0800 Subject: [PATCH 11/11] Address the comments Change-Id: I09c2b26661f42334c095303ec8c76e821d4145d3 --- .../spark/status/api/v1/AllRDDResource.scala | 14 ++++---- .../org/apache/spark/status/api/v1/api.scala | 8 ++--- .../spark/storage/BlockManagerSource.scala | 4 +-- .../spark/storage/StorageStatusListener.scala | 2 ++ .../apache/spark/storage/StorageUtils.scala | 28 +++++++++------- .../apache/spark/ui/exec/ExecutorsPage.scala | 5 ++- .../org/apache/spark/ui/storage/RDDPage.scala | 8 ++--- .../apache/spark/storage/StorageSuite.scala | 12 +++---- .../org/apache/spark/ui/UISeleniumSuite.scala | 32 ++++++++++++------- 9 files changed, 62 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala index b352f84eb39b0..1279b281ad8d8 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala @@ -71,14 +71,12 @@ private[spark] object AllRDDResource { memoryUsed = status.memUsedByRdd(rddId), memoryRemaining = status.memRemaining, diskUsed = status.diskUsedByRdd(rddId), - onHeapMemoryUsed = - if (!rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L, - offHeapMemoryUsed = - if (rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L, - // Get RDDStorageInfo REST API will return RDD information only in the Live UI, so - // we assume this two fields always exist. - onHeapMemoryRemaining = status.onHeapMemRemaining.get, - offHeapMemoryRemaining = status.offHeapMemRemaining.get + onHeapMemoryUsed = Some( + if (!rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L), + offHeapMemoryUsed = Some( + if (rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L), + onHeapMemoryRemaining = status.onHeapMemRemaining, + offHeapMemoryRemaining = status.offHeapMemRemaining ) } ) } else { None diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 9b63c89ad77b3..d159b9450ef5c 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -116,10 +116,10 @@ class RDDDataDistribution private[spark]( val memoryUsed: Long, val memoryRemaining: Long, val diskUsed: Long, - val onHeapMemoryUsed: Long, - val offHeapMemoryUsed: Long, - val onHeapMemoryRemaining: Long, - val offHeapMemoryRemaining: Long) + val onHeapMemoryUsed: Option[Long], + val offHeapMemoryUsed: Option[Long], + val onHeapMemoryRemaining: Option[Long], + val offHeapMemoryRemaining: Option[Long]) class RDDPartitionInfo private[spark]( val blockName: String, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index b6cb041138e9d..197a01762c0c5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -54,10 +54,10 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) _.getStorageStatus.map(_.memUsed).sum) registerGauge(MetricRegistry.name("memory", "onHeapMemUsed_MB"), - _.getStorageStatus.map(_.onHeapMemUsed).sum) + _.getStorageStatus.map(_.onHeapMemUsed.getOrElse(0L)).sum) registerGauge(MetricRegistry.name("memory", "offHeapMemUsed_MB"), - _.getStorageStatus.map(_.offHeapMemUsed).sum) + _.getStorageStatus.map(_.offHeapMemUsed.getOrElse(0L)).sum) registerGauge(MetricRegistry.name("disk", "diskSpaceUsed_MB"), _.getStorageStatus.map(_.diskUsed).sum) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index b5faed81350db..1b30d4fa93bc0 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -74,6 +74,8 @@ class StorageStatusListener(conf: SparkConf) extends SparkListener { synchronized { val blockManagerId = blockManagerAdded.blockManagerId val executorId = blockManagerId.executorId + // The onHeap and offHeap memory are always defined for new applications, + // but they can be missing if we are replaying old event logs. val storageStatus = new StorageStatus(blockManagerId, blockManagerAdded.maxMem, blockManagerAdded.maxOnHeapMem, blockManagerAdded.maxOffHeapMem) executorIdToStorageStatus(executorId) = storageStatus diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 840966bed7d0e..6bcbb5b46ebcb 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -37,8 +37,6 @@ import org.apache.spark.internal.Logging @DeveloperApi class StorageStatus( val blockManagerId: BlockManagerId, - // Explicitly adding this maxMemory field to handle maxOnHeapMem and maxOffHeapMem not - // existing issue, this happened when trying to replay old event log. val maxMemory: Long, val maxOnHeapMem: Option[Long], val maxOffHeapMem: Option[Long]) { @@ -185,32 +183,38 @@ class StorageStatus( def memRemaining: Long = maxMem - memUsed /** Return the memory used by caching RDDs */ - def cacheSize: Long = onHeapCacheSize + offHeapCacheSize + def cacheSize: Long = onHeapCacheSize.getOrElse(0L) + offHeapCacheSize.getOrElse(0L) /** Return the memory used by this block manager. */ - def memUsed: Long = onHeapMemUsed + offHeapMemUsed + def memUsed: Long = onHeapMemUsed.getOrElse(0L) + offHeapMemUsed.getOrElse(0L) /** Return the on-heap memory remaining in this block manager. */ - def onHeapMemRemaining: Option[Long] = maxOnHeapMem.map(_ - onHeapMemUsed) + def onHeapMemRemaining: Option[Long] = + for (m <- maxOnHeapMem; o <- onHeapMemUsed) yield m - o /** Return the off-heap memory remaining in this block manager. */ - def offHeapMemRemaining: Option[Long] = maxOffHeapMem.map(_ - offHeapMemUsed) + def offHeapMemRemaining: Option[Long] = + for (m <- maxOffHeapMem; o <- offHeapMemUsed) yield m - o /** Return the on-heap memory used by this block manager. */ - def onHeapMemUsed: Long = _nonRddStorageInfo.onHeapUsage + onHeapCacheSize + def onHeapMemUsed: Option[Long] = onHeapCacheSize.map(_ + _nonRddStorageInfo.onHeapUsage) /** Return the off-heap memory used by this block manager. */ - def offHeapMemUsed: Long = _nonRddStorageInfo.offHeapUsage + offHeapCacheSize + def offHeapMemUsed: Option[Long] = offHeapCacheSize.map(_ + _nonRddStorageInfo.offHeapUsage) /** Return the memory used by on-heap caching RDDs */ - def onHeapCacheSize: Long = _rddStorageInfo.collect { + def onHeapCacheSize: Option[Long] = maxOnHeapMem.map { _ => + _rddStorageInfo.collect { case (_, storageInfo) if !storageInfo.level.useOffHeap => storageInfo.memoryUsage - }.sum + }.sum + } /** Return the memory used by off-heap caching RDDs */ - def offHeapCacheSize: Long = _rddStorageInfo.collect { + def offHeapCacheSize: Option[Long] = maxOffHeapMem.map { _ => + _rddStorageInfo.collect { case (_, storageInfo) if storageInfo.level.useOffHeap => storageInfo.memoryUsage - }.sum + }.sum + } /** Return the disk space used by this block manager. */ def diskUsed: Long = _nonRddStorageInfo.diskUsage + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 494b69c3412cc..887fd17b35562 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -115,9 +115,8 @@ private[spark] object ExecutorsPage { val rddBlocks = status.numBlocks val memUsed = status.memUsed val maxMem = status.maxMem - // Only maxOnHeapMem and maxOffHeapMem are defined these two fields are not None. - val onHeapMemUsed = status.maxOnHeapMem.map(_ => status.onHeapMemUsed) - val offHeapMemUsed = status.maxOffHeapMem.map(_ => status.offHeapMemUsed) + val onHeapMemUsed = status.onHeapMemUsed + val offHeapMemUsed = status.offHeapMemUsed val maxOnHeapMem = status.maxOnHeapMem val maxOffHeapMem = status.maxOffHeapMem val diskUsed = status.diskUsed diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index b07ff4d302840..a1a0c729b9240 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -156,12 +156,12 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { {worker.address} - {Utils.bytesToString(worker.onHeapMemoryUsed)} - ({Utils.bytesToString(worker.onHeapMemoryRemaining)} Remaining) + {Utils.bytesToString(worker.onHeapMemoryUsed.getOrElse(0L))} + ({Utils.bytesToString(worker.onHeapMemoryRemaining.getOrElse(0L))} Remaining) - {Utils.bytesToString(worker.offHeapMemoryUsed)} - ({Utils.bytesToString(worker.offHeapMemoryRemaining)} Remaining) + {Utils.bytesToString(worker.offHeapMemoryUsed.getOrElse(0L))} + ({Utils.bytesToString(worker.offHeapMemoryRemaining.getOrElse(0L))} Remaining) {Utils.bytesToString(worker.diskUsed)} diff --git a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala index 5bdd81fab048b..da198f946fd64 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -362,8 +362,8 @@ class StorageSuite extends SparkFunSuite { assert(status.memUsed === actualMemUsed) assert(status.diskUsed === actualDiskUsed) - assert(status.onHeapMemUsed === actualOnHeapMemUsed) - assert(status.offHeapMemUsed === actualOffHeapMemUsed) + assert(status.onHeapMemUsed.get === actualOnHeapMemUsed) + assert(status.offHeapMemUsed.get === actualOffHeapMemUsed) assert(status.memRemaining === status.maxMem - actualMemUsed) assert(status.onHeapMemRemaining.get === status.maxOnHeapMem.get - actualOnHeapMemUsed) @@ -379,8 +379,8 @@ class StorageSuite extends SparkFunSuite { status.updateBlock(RDDBlockId(1, 1), BlockStatus(offheap, 4L, 0L)) assert(status.memUsed === actualMemUsed) assert(status.diskUsed === actualDiskUsed) - assert(status.onHeapMemUsed === actualOnHeapMemUsed) - assert(status.offHeapMemUsed === actualOffHeapMemUsed) + assert(status.onHeapMemUsed.get === actualOnHeapMemUsed) + assert(status.offHeapMemUsed.get === actualOffHeapMemUsed) status.removeBlock(TestBlockId("fire")) status.removeBlock(TestBlockId("man")) @@ -402,8 +402,8 @@ class StorageSuite extends SparkFunSuite { assert(status.memUsed === 0L) assert(status.diskUsed === 0L) - assert(status.onHeapMemUsed === 0L) - assert(status.offHeapMemUsed === 0L) + assert(status.onHeapMemUsed === None) + assert(status.offHeapMemUsed === None) assert(status.memRemaining === status.maxMem) assert(status.onHeapMemRemaining === None) diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index f65a40aefd964..f4c561c737794 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -156,12 +156,16 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val dataDistributions0 = (updatedRddJson \ "dataDistribution").extract[Seq[RDDDataDistribution]] dataDistributions0.length should be (1) - val distribution0 = dataDistributions0.head - distribution0.memoryUsed should be (distribution0.onHeapMemoryUsed) - distribution0.memoryRemaining should be ( - distribution0.onHeapMemoryRemaining + distribution0.offHeapMemoryRemaining) - distribution0.onHeapMemoryUsed should not be (0L) - distribution0.offHeapMemoryUsed should be (0L) + val dist0 = dataDistributions0.head + + dist0.onHeapMemoryUsed should not be (None) + dist0.memoryUsed should be (dist0.onHeapMemoryUsed.get) + dist0.onHeapMemoryRemaining should not be (None) + dist0.offHeapMemoryRemaining should not be (None) + dist0.memoryRemaining should be ( + dist0.onHeapMemoryRemaining.get + dist0.offHeapMemoryRemaining.get) + dist0.onHeapMemoryUsed should not be (Some(0L)) + dist0.offHeapMemoryUsed should be (Some(0L)) rdd.unpersist() rdd.persist(StorageLevels.OFF_HEAP).count() @@ -171,12 +175,16 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val dataDistributions1 = (updatedRddJson1 \ "dataDistribution").extract[Seq[RDDDataDistribution]] dataDistributions1.length should be (1) - val distribution1 = dataDistributions1.head - distribution1.memoryUsed should be (distribution1.offHeapMemoryUsed) - distribution1.memoryRemaining should be ( - distribution1.onHeapMemoryRemaining + distribution1.offHeapMemoryRemaining) - distribution1.onHeapMemoryUsed should be (0L) - distribution1.offHeapMemoryUsed should not be (0L) + val dist1 = dataDistributions1.head + + dist1.offHeapMemoryUsed should not be (None) + dist1.memoryUsed should be (dist1.offHeapMemoryUsed.get) + dist1.onHeapMemoryRemaining should not be (None) + dist1.offHeapMemoryRemaining should not be (None) + dist1.memoryRemaining should be ( + dist1.onHeapMemoryRemaining.get + dist1.offHeapMemoryRemaining.get) + dist1.onHeapMemoryUsed should be (Some(0L)) + dist1.offHeapMemoryUsed should not be (Some(0L)) } }