From 53af15d25e19b8b63bcad035e3149e2920943561 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 30 Jul 2014 16:20:40 -0700 Subject: [PATCH 01/16] Refactor StorageStatus + add a bunch of tests This commit refactors storage status to keep around a set of RDD IDs which have blocks stored in the status' block manager. The purpose is such that we don't have to linearly scan through every single storage status' blocks if it doesn't even contain blocks for the RDD we're interested in in the first place. This commit also adds a bunch of tests for StorageStatus and StorageUtils methods. There were previously a few minor bugs in StorageUtils.blockLocationsFromStorageStatus and StorageUtils.filterStorageStatusByRDD that are now fixed and tested. Going forward, we need to first cleanup the method signatures to reflect what they actually do. Then we will make things more efficient now that we've set the stage. --- .../storage/BlockManagerMasterActor.scala | 5 +- .../spark/storage/BlockManagerSource.scala | 6 +- .../spark/storage/StorageStatusListener.scala | 9 +- .../apache/spark/storage/StorageUtils.scala | 115 ++++--- .../apache/spark/ui/exec/ExecutorsPage.scala | 2 +- .../apache/spark/ui/storage/StorageTab.scala | 4 +- .../apache/spark/storage/StorageSuite.scala | 292 ++++++++++++++++++ 7 files changed, 380 insertions(+), 53 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/StorageSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index de1cc5539fb4..c9b4657887f5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -265,8 +265,9 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private def storageStatus: Array[StorageStatus] = { blockManagerInfo.map { case(blockManagerId, info) => - val blockMap = mutable.Map[BlockId, BlockStatus](info.blocks.toSeq: _*) - new StorageStatus(blockManagerId, info.maxMem, blockMap) + val storageStatus = new StorageStatus(blockManagerId, info.maxMem) + info.blocks.foreach { case (id, status) => storageStatus.addBlock(id, status) } + storageStatus }.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 687586490abf..760c508f3f09 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -55,11 +55,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList - .flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_ + _) - .getOrElse(0L) - + val diskSpaceUsed = storageStatusList.map(_.diskUsed).reduceOption(_ + _).getOrElse(0L) diskSpaceUsed / 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 41c960c867e2..7b75afddef68 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -39,9 +39,9 @@ class StorageStatusListener extends SparkListener { filteredStatus.foreach { storageStatus => updatedBlocks.foreach { case (blockId, updatedStatus) => if (updatedStatus.storageLevel == StorageLevel.NONE) { - storageStatus.blocks.remove(blockId) + storageStatus.removeBlock(blockId) } else { - storageStatus.blocks(blockId) = updatedStatus + storageStatus.updateBlock(blockId, updatedStatus) } } } @@ -50,9 +50,8 @@ class StorageStatusListener extends SparkListener { /** Update storage status list to reflect the removal of an RDD from the cache */ private def updateStorageStatus(unpersistedRDDId: Int) { storageStatusList.foreach { storageStatus => - val unpersistedBlocksIds = storageStatus.rddBlocks.keys.filter(_.rddId == unpersistedRDDId) - unpersistedBlocksIds.foreach { blockId => - storageStatus.blocks.remove(blockId) + storageStatus.rddBlocks(unpersistedRDDId).foreach { case (blockId, _) => + storageStatus.removeBlock(blockId) } } } 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 177281f66336..193c435e4a05 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -28,55 +28,96 @@ import org.apache.spark.annotation.DeveloperApi * Storage information for each BlockManager. */ @DeveloperApi -class StorageStatus( - val blockManagerId: BlockManagerId, - val maxMem: Long, - val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { +class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { + private val _blocks = new mutable.HashMap[BlockId, BlockStatus] + private val _rddIds = new mutable.HashSet[Int] + + /** Return the blocks stored in this block manager as a mapping from ID to status. */ + def blocks: Map[BlockId, BlockStatus] = _blocks + + /** Add the given block, keeping track of the RDD ID if this is an RDD block. */ + def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { + blockId match { + case RDDBlockId(rddId, _) => _rddIds.add(rddId) + case _ => + } + _blocks(blockId) = blockStatus + } - def memUsed = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + /** Update the given block, keeping track of the RDD ID if this is an RDD block. */ + def updateBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = addBlock(blockId, blockStatus) + + /** Remove the given block, keeping track of the RDD ID if this is an RDD block. */ + def removeBlock(blockId: BlockId): Option[BlockStatus] = { + val removed = _blocks.remove(blockId) + blockId match { + case RDDBlockId(rddId, _) => + if (rddBlocks(rddId).isEmpty) { + _rddIds.remove(rddId) + } + case _ => + } + removed + } - def memUsedByRDD(rddId: Int) = - rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + /** Return the IDs of the RDDs which have blocks stored in this block manager. */ + def rddIds: Seq[Int] = _rddIds.toSeq - def diskUsed = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + /** Return the RDD blocks stored in this block manager as a mapping from ID to status. */ + def rddBlocks: Map[RDDBlockId, BlockStatus] = + blocks.filterKeys(_.isInstanceOf[RDDBlockId]).asInstanceOf[Map[RDDBlockId, BlockStatus]] + + /** + * Return the RDD blocks with the given RDD ID stored in this block manager as a mapping + * from ID to status. + */ + def rddBlocks(rddId: Int): Map[RDDBlockId, BlockStatus] = rddBlocks.filterKeys(_.rddId == rddId) - def diskUsedByRDD(rddId: Int) = - rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + /** Return the memory used by this block manager. */ + def memUsed: Long = memUsed(blocks.values) + /** Return the memory used by the given RDD in this block manager. */ + def memUsedByRDD(rddId: Int): Long = memUsed(rddBlocks(rddId).values) + + /** Return the memory remaining in this block manager. */ def memRemaining: Long = maxMem - memUsed - def rddBlocks = blocks.collect { case (rdd: RDDBlockId, status) => (rdd, status) } + /** Return the disk space used by this block manager. */ + def diskUsed: Long = diskUsed(blocks.values) + + /** Return the disk space used by the given RDD in this block manager. */ + def diskUsedByRDD(rddId: Int): Long = diskUsed(rddBlocks(rddId).values) + + // Helper methods for computing memory and disk usages + private def memUsed(statuses: Iterable[BlockStatus]): Long = + statuses.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + private def diskUsed(statuses: Iterable[BlockStatus]): Long = + statuses.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) } /** Helper methods for storage-related objects. */ private[spark] object StorageUtils { - /** - * Returns basic information of all RDDs persisted in the given SparkContext. This does not - * include storage information. - */ - def rddInfoFromSparkContext(sc: SparkContext): Array[RDDInfo] = { - sc.persistentRdds.values.map { rdd => + /** Returns storage information of all RDDs persisted in the given SparkContext. */ + def rddInfoFromStorageStatus( + storageStatuses: Seq[StorageStatus], + sc: SparkContext): Array[RDDInfo] = { + val rddInfos = sc.persistentRdds.values.map { rdd => val rddName = Option(rdd.name).getOrElse(rdd.id.toString) val rddNumPartitions = rdd.partitions.size val rddStorageLevel = rdd.getStorageLevel val rddInfo = new RDDInfo(rdd.id, rddName, rddNumPartitions, rddStorageLevel) rddInfo }.toArray - } - - /** Returns storage information of all RDDs persisted in the given SparkContext. */ - def rddInfoFromStorageStatus( - storageStatuses: Seq[StorageStatus], - sc: SparkContext): Array[RDDInfo] = { - rddInfoFromStorageStatus(storageStatuses, rddInfoFromSparkContext(sc)) + rddInfoFromStorageStatus(storageStatuses, rddInfos) + rddInfos } /** Returns storage information of all RDDs in the given list. */ def rddInfoFromStorageStatus( storageStatuses: Seq[StorageStatus], rddInfos: Seq[RDDInfo], - updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Array[RDDInfo] = { + updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Unit = { // Mapping from a block ID -> its status val blockMap = mutable.Map(storageStatuses.flatMap(_.rddBlocks): _*) @@ -94,7 +135,7 @@ private[spark] object StorageUtils { // Mapping from RDD ID -> the associated RDDInfo (with potentially outdated storage information) val rddInfoMap = rddInfos.map { info => (info.id, info) }.toMap - val rddStorageInfos = rddBlockMap.flatMap { case (rddId, blocks) => + rddBlockMap.foreach { case (rddId, blocks) => // Add up memory, disk and Tachyon sizes val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize + status.tachyonSize > 0 } @@ -111,31 +152,31 @@ private[spark] object StorageUtils { rddInfo.tachyonSize = tachyonSize rddInfo } - }.toArray - - scala.util.Sorting.quickSort(rddStorageInfos) - rddStorageInfos + } } /** Returns a mapping from BlockId to the locations of the associated block. */ def blockLocationsFromStorageStatus( storageStatuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { + // An ungrouped list of (blockId, location) pairs val blockLocationPairs = storageStatuses.flatMap { storageStatus => storageStatus.blocks.map { case (bid, _) => (bid, storageStatus.blockManagerId.hostPort) } } - blockLocationPairs.toMap + blockLocationPairs .groupBy { case (blockId, _) => blockId } - .mapValues(_.values.toSeq) + .mapValues { rddLocations => rddLocations.map { case (_, location) => location } } } /** Filters the given list of StorageStatus by the given RDD ID. */ def filterStorageStatusByRDD( storageStatuses: Seq[StorageStatus], rddId: Int): Array[StorageStatus] = { - storageStatuses.map { status => - val filteredBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toSeq - val filteredBlockMap = mutable.Map[BlockId, BlockStatus](filteredBlocks: _*) - new StorageStatus(status.blockManagerId, status.maxMem, filteredBlockMap) - }.toArray + storageStatuses + .filter(_.rddIds.contains(rddId)) + .map { status => + val newStatus = new StorageStatus(status.blockManagerId, status.maxMem) + status.rddBlocks(rddId).foreach { case (bid, bstatus) => newStatus.addBlock(bid, bstatus) } + newStatus + }.toArray } } 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 b358c855e1c8..46d4d68efbf2 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 @@ -51,7 +51,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) val memUsed = storageStatusList.map(_.memUsed).fold(0L)(_ + _) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) + val diskSpaceUsed = storageStatusList.map(_.diskUsed).reduceOption(_ + _).getOrElse(0L) val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.id) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 0cc0cf311717..73242518756b 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -51,9 +51,7 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar /** Update each RDD's info to reflect any updates to the RDD's storage status */ private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty) { val rddInfos = _rddInfoMap.values.toSeq - val updatedRddInfos = - StorageUtils.rddInfoFromStorageStatus(storageStatusList, rddInfos, updatedBlocks) - updatedRddInfos.foreach { info => _rddInfoMap(info.id) = info } + StorageUtils.rddInfoFromStorageStatus(storageStatusList, rddInfos, updatedBlocks) } /** diff --git a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala new file mode 100644 index 000000000000..a265a081a4bf --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -0,0 +1,292 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.scalatest.FunSuite + +/** + * Test various functionalities in StorageUtils and StorageStatus. + */ +class StorageSuite extends FunSuite { + private val memAndDisk = StorageLevel.MEMORY_AND_DISK + + // A list of StorageStatuses with RDDs 0 and 1 cached on different block managers + private def stockStorageStatuses: Seq[StorageStatus] = { + val status1 = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) + val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2, 2), 2000L) + val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3, 3), 3000L) + status1.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status1.addBlock(RDDBlockId(0, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(0, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(0, 3), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status3.addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status3.addBlock(RDDBlockId(1, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) + Seq(status1, status2, status3) + } + + // A list of RDDInfo for RDDs 0 and 1 + private def stockRDDInfos: Seq[RDDInfo] = { + val info0 = new RDDInfo(0, "0", 10, memAndDisk) + val info1 = new RDDInfo(1, "1", 3, memAndDisk) + Seq(info0, info1) + } + + test("storage status add/remove blocks") { + val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) + assert(status.blocks.isEmpty) + assert(status.rddBlocks.isEmpty) + assert(status.rddIds.isEmpty) + assert(status.memUsed === 0) + assert(status.memRemaining === 1000L) + assert(status.diskUsed === 0) + + // Add a few blocks + status.addBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(TestBlockId("faa"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + assert(status.blocks.size === 3) + assert(status.rddBlocks.isEmpty) + assert(status.rddIds.isEmpty) + assert(status.memUsed === 30L) + assert(status.memRemaining === 970L) + assert(status.diskUsed === 60L) + + // Update a few existing blocks + status.updateBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 50L, 100L, 0L)) + status.updateBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 100L, 20L, 0L)) + assert(status.blocks.size === 3) + assert(status.rddBlocks.isEmpty) + assert(status.rddIds.isEmpty) + assert(status.memUsed === 160L) + assert(status.memRemaining === 840L) + assert(status.diskUsed === 140L) + + // Remove a few blocks + status.removeBlock(TestBlockId("foo")) + status.removeBlock(TestBlockId("faa")) + assert(status.blocks.size === 1) + assert(status.rddBlocks.isEmpty) + assert(status.rddIds.isEmpty) + assert(status.memUsed === 100L) + assert(status.memRemaining === 900L) + assert(status.diskUsed === 20L) + } + + test("storage status add/remove RDD blocks") { + val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) + assert(status.blocks.isEmpty) + assert(status.rddBlocks.isEmpty) + assert(status.rddIds.isEmpty) + + // Add a few blocks + status.addBlock(TestBlockId("DANGER"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(TestBlockId("MANGER"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 100L, 200L, 0L)) + status.addBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(RDDBlockId(2, 3), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(RDDBlockId(2, 4), BlockStatus(memAndDisk, 10L, 40L, 0L)) + assert(status.blocks.size === 7) + assert(status.rddBlocks.size === 5) + assert(status.rddIds.toSet === Seq(0, 1, 2).toSet) + assert(status.rddBlocks(0).size === 1) + assert(status.rddBlocks(0).head._1 === RDDBlockId(0, 0)) + assert(status.rddBlocks(0).head._2 === BlockStatus(memAndDisk, 10L, 20L, 0L)) + assert(status.rddBlocks(1).size === 1) + assert(status.rddBlocks(1).head._1 === RDDBlockId(1, 1)) + assert(status.rddBlocks(1).head._2 === BlockStatus(memAndDisk, 100L, 200L, 0L)) + assert(status.rddBlocks(2).size === 3) + assert(status.rddBlocks(2).head._1 === RDDBlockId(2, 2)) + assert(status.rddBlocks(2).head._2 === BlockStatus(memAndDisk, 10L, 20L, 0L)) + assert(status.memUsedByRDD(0) === 10L) + assert(status.memUsedByRDD(1) === 100L) + assert(status.memUsedByRDD(2) === 30L) + assert(status.diskUsedByRDD(0) === 20L) + assert(status.diskUsedByRDD(1) === 200L) + assert(status.diskUsedByRDD(2) === 80L) + + // Update a few blocks + status.addBlock(TestBlockId("DANGER"), BlockStatus(memAndDisk, 5000L, 0L, 0L)) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)) + status.updateBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 0L, 1000L, 0L)) + assert(status.rddBlocks(0).size === 1) + assert(status.rddBlocks(0).head._1 === RDDBlockId(0, 0)) + assert(status.rddBlocks(0).head._2 === BlockStatus(memAndDisk, 0L, 0L, 0L)) + assert(status.rddBlocks(2).size === 3) + assert(status.rddBlocks(2).head._1 === RDDBlockId(2, 2)) + assert(status.rddBlocks(2).head._2 === BlockStatus(memAndDisk, 0L, 1000L, 0L)) + assert(status.memUsedByRDD(0) === 0L) + assert(status.memUsedByRDD(1) === 100L) + assert(status.memUsedByRDD(2) === 20L) + assert(status.diskUsedByRDD(0) === 0L) + assert(status.diskUsedByRDD(1) === 200L) + assert(status.diskUsedByRDD(2) === 1060L) + + // Remove a few blocks + status.removeBlock(TestBlockId("MANGER")) + status.removeBlock(RDDBlockId(1, 1)) + status.removeBlock(RDDBlockId(2, 2)) + status.removeBlock(RDDBlockId(2, 4)) + assert(status.blocks.size === 3) + assert(status.rddBlocks.size === 2) + assert(status.rddIds.toSet === Seq(0, 2).toSet) + assert(status.rddBlocks(0).size === 1) + assert(status.rddBlocks(1).size === 0) + assert(status.rddBlocks(2).size === 1) + assert(status.memUsedByRDD(0) === 0L) + assert(status.memUsedByRDD(1) === 0L) + assert(status.memUsedByRDD(2) === 10L) + assert(status.diskUsedByRDD(0) === 0L) + assert(status.diskUsedByRDD(1) === 0L) + assert(status.diskUsedByRDD(2) === 20L) + + // Add a few blocks again + status.addBlock(RDDBlockId(2, 10), BlockStatus(memAndDisk, 10L, 200000L, 0L)) + status.addBlock(RDDBlockId(3, 5), BlockStatus(memAndDisk, 10L, 200L, 0L)) + assert(status.blocks.size === 5) + assert(status.rddBlocks.size === 4) + assert(status.rddIds.toSet === Seq(0, 2, 3).toSet) + assert(status.rddBlocks(0).size === 1) + assert(status.rddBlocks(1).size === 0) + assert(status.rddBlocks(2).size === 2) + assert(status.rddBlocks(3).size === 1) + assert(status.rddBlocks(3).head._1 === RDDBlockId(3, 5)) + assert(status.rddBlocks(3).head._2 === BlockStatus(memAndDisk, 10L, 200L, 0L)) + assert(status.memUsedByRDD(0) === 0L) + assert(status.memUsedByRDD(1) === 0L) + assert(status.memUsedByRDD(2) === 20L) + assert(status.memUsedByRDD(3) === 10L) + assert(status.diskUsedByRDD(0) === 0L) + assert(status.diskUsedByRDD(1) === 0L) + assert(status.diskUsedByRDD(2) === 200020L) + assert(status.diskUsedByRDD(3) === 200L) + } + + test("StorageUtils.rddInfoFromStorageStatus") { + val storageStatuses = stockStorageStatuses + val rddInfos = stockRDDInfos + StorageUtils.rddInfoFromStorageStatus(storageStatuses, rddInfos) + assert(rddInfos(0).numCachedPartitions === 5) + assert(rddInfos(0).memSize === 5L) + assert(rddInfos(0).diskSize === 10L) + assert(rddInfos(1).numCachedPartitions === 3) + assert(rddInfos(1).memSize === 3L) + assert(rddInfos(1).diskSize === 6L) + } + + test("StorageUtils.rddInfoFromStorageStatus with updated blocks") { + val storageStatuses = stockStorageStatuses + val rddInfos = stockRDDInfos + + // Drop 3 blocks from RDD 0, and cache more of RDD 1 + val updatedBlocks1 = Seq( + (RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(0, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(0, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(1, 0), BlockStatus(memAndDisk, 100L, 100L, 0L)), + (RDDBlockId(1, 100), BlockStatus(memAndDisk, 100L, 100L, 0L)) + ) + StorageUtils.rddInfoFromStorageStatus(storageStatuses, rddInfos, updatedBlocks1) + assert(rddInfos(0).numCachedPartitions === 2) + assert(rddInfos(0).memSize === 2L) + assert(rddInfos(0).diskSize === 4L) + assert(rddInfos(1).numCachedPartitions === 4) + assert(rddInfos(1).memSize === 202L) + assert(rddInfos(1).diskSize === 204L) + + // Actually update storage statuses so we can chain the calls to rddInfoFromStorageStatus + updatedBlocks1.foreach { case (bid, bstatus) => + val statusWithBlock = storageStatuses.find(_.blocks.contains(bid)) + statusWithBlock match { + case Some(s) => s.updateBlock(bid, bstatus) + case None => storageStatuses(0).addBlock(bid, bstatus) // arbitrarily pick the first + } + } + + // Drop all of RDD 1 + val updatedBlocks2 = Seq( + (RDDBlockId(1, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(1, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(1, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(1, 100), BlockStatus(memAndDisk, 0L, 0L, 0L)) + ) + StorageUtils.rddInfoFromStorageStatus(storageStatuses, rddInfos, updatedBlocks2) + assert(rddInfos(0).numCachedPartitions === 2) + assert(rddInfos(0).memSize === 2L) + assert(rddInfos(0).diskSize === 4L) + assert(rddInfos(1).numCachedPartitions === 0) + assert(rddInfos(1).memSize === 0L) + assert(rddInfos(1).diskSize === 0L) + } + + test("StorageUtils.blockLocationsFromStorageStatus") { + val storageStatuses = stockStorageStatuses + val blockLocations1 = StorageUtils.blockLocationsFromStorageStatus(storageStatuses) + assert(blockLocations1.contains(RDDBlockId(0, 0))) + assert(blockLocations1.contains(RDDBlockId(0, 1))) + assert(blockLocations1.contains(RDDBlockId(0, 2))) + assert(blockLocations1.contains(RDDBlockId(0, 3))) + assert(blockLocations1.contains(RDDBlockId(0, 4))) + assert(blockLocations1.contains(RDDBlockId(1, 0))) + assert(blockLocations1.contains(RDDBlockId(1, 1))) + assert(blockLocations1.contains(RDDBlockId(1, 2))) + assert(blockLocations1.size === 8) + assert(blockLocations1(RDDBlockId(0, 0)) === Seq("dog:1")) + assert(blockLocations1(RDDBlockId(0, 1)) === Seq("dog:1")) + assert(blockLocations1(RDDBlockId(0, 2)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(0, 3)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(1, 0)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(1, 1)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(0, 4)) === Seq("cat:3")) + assert(blockLocations1(RDDBlockId(1, 2)) === Seq("cat:3")) + + // Multiple locations + storageStatuses(0).addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + storageStatuses(0).addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) + storageStatuses(2).addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + val blockLocations2 = StorageUtils.blockLocationsFromStorageStatus(storageStatuses) + assert(blockLocations2.contains(RDDBlockId(0, 0))) + assert(blockLocations2.contains(RDDBlockId(0, 1))) + assert(blockLocations2.contains(RDDBlockId(0, 2))) + assert(blockLocations2.contains(RDDBlockId(0, 3))) + assert(blockLocations2.contains(RDDBlockId(0, 4))) + assert(blockLocations2.contains(RDDBlockId(1, 0))) + assert(blockLocations2.contains(RDDBlockId(1, 1))) + assert(blockLocations2.contains(RDDBlockId(1, 2))) + assert(blockLocations2.size === 8) + assert(blockLocations2(RDDBlockId(0, 0)) === Seq("dog:1", "cat:3")) + assert(blockLocations2(RDDBlockId(0, 1)) === Seq("dog:1")) + assert(blockLocations2(RDDBlockId(0, 2)) === Seq("duck:2")) + assert(blockLocations2(RDDBlockId(0, 3)) === Seq("duck:2")) + assert(blockLocations2(RDDBlockId(1, 0)) === Seq("dog:1", "duck:2")) + assert(blockLocations2(RDDBlockId(1, 1)) === Seq("duck:2")) + assert(blockLocations2(RDDBlockId(0, 4)) === Seq("dog:1", "cat:3")) + assert(blockLocations2(RDDBlockId(1, 2)) === Seq("cat:3")) + } + + test("StorageUtils.filterStorageStatusByRDD") { + val storageStatuses = stockStorageStatuses + val filteredStorageStatuses0 = StorageUtils.filterStorageStatusByRDD(storageStatuses, 0) + val filteredStorageStatuses1 = StorageUtils.filterStorageStatusByRDD(storageStatuses, 1) + assert(filteredStorageStatuses0.size === 3) + assert(filteredStorageStatuses1.size === 2) + } + +} From 41fa50df1fc520802905b2f716b2008004c7c79d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 30 Jul 2014 18:51:56 -0700 Subject: [PATCH 02/16] Add a legacy constructor for StorageStatus This just makes it easier to create one with a source of blocks. --- .../storage/BlockManagerMasterActor.scala | 6 ++---- .../apache/spark/storage/StorageUtils.scala | 19 +++++++++++++++---- 2 files changed, 17 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index c9b4657887f5..f76ab323d3ef 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -264,10 +264,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } private def storageStatus: Array[StorageStatus] = { - blockManagerInfo.map { case(blockManagerId, info) => - val storageStatus = new StorageStatus(blockManagerId, info.maxMem) - info.blocks.foreach { case (id, status) => storageStatus.addBlock(id, status) } - storageStatus + blockManagerInfo.map { case (blockManagerId, info) => + new StorageStatus(blockManagerId, info.maxMem, info.blocks) }.toArray } 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 193c435e4a05..9eeb48a69075 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -25,13 +25,23 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: - * Storage information for each BlockManager. + * Storage information for each BlockManager. This class assumes BlockId and BlockStatus are + * immutable, such that the consumers of this class will not mutate the source of the information. */ @DeveloperApi class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { private val _blocks = new mutable.HashMap[BlockId, BlockStatus] private val _rddIds = new mutable.HashSet[Int] + /** + * Instantiate a StorageStatus with the given initial blocks. This essentially makes a copy of + * the original blocks map such that the fate of this storage status is not tied to the source. + */ + def this(bmid: BlockManagerId, maxMem: Long, initialBlocks: Map[BlockId, BlockStatus]) { + this(bmid, maxMem) + initialBlocks.foreach { case (blockId, blockStatus) => addBlock(blockId, blockStatus) } + } + /** Return the blocks stored in this block manager as a mapping from ID to status. */ def blocks: Map[BlockId, BlockStatus] = _blocks @@ -174,9 +184,10 @@ private[spark] object StorageUtils { storageStatuses .filter(_.rddIds.contains(rddId)) .map { status => - val newStatus = new StorageStatus(status.blockManagerId, status.maxMem) - status.rddBlocks(rddId).foreach { case (bid, bstatus) => newStatus.addBlock(bid, bstatus) } - newStatus + new StorageStatus( + status.blockManagerId, + status.maxMem, + status.rddBlocks(rddId).asInstanceOf[Map[BlockId, BlockStatus]]) }.toArray } } From 7b2c4aae86c784e117809fd857c31a3a402dd958 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 30 Jul 2014 19:18:52 -0700 Subject: [PATCH 03/16] Rewrite blockLocationsFromStorageStatus + clean up method signatures The existing implementation of blockLocationFromStorageStatus relies on a groupBy, which is somewhat expensive. The new code creates a map from the get go and adds the block locations by iterating through the storage statuses' blocks. This commit also cleans up StorageUtils method signatures by removing unnecessary methods and renaming others with long-winded names. --- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../apache/spark/storage/StorageUtils.scala | 64 +++++------ .../org/apache/spark/ui/storage/RDDPage.scala | 4 +- .../apache/spark/ui/storage/StorageTab.scala | 7 +- .../apache/spark/storage/StorageSuite.scala | 104 +++++++++--------- 5 files changed, 87 insertions(+), 96 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fb4c86716bb8..8b1916d7badf 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -840,7 +840,9 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi def getRDDStorageInfo: Array[RDDInfo] = { - StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) + val rddInfos = StorageUtils.makeRddInfo(this) + StorageUtils.updateRddInfo(getExecutorStorageStatus, rddInfos) + rddInfos.toArray } /** 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 9eeb48a69075..06727637d22f 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -30,6 +30,8 @@ import org.apache.spark.annotation.DeveloperApi */ @DeveloperApi class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { + + // This should not be mutated directly, but through the add/update/removeBlock methods private val _blocks = new mutable.HashMap[BlockId, BlockStatus] private val _rddIds = new mutable.HashSet[Int] @@ -108,35 +110,22 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** Helper methods for storage-related objects. */ private[spark] object StorageUtils { - /** Returns storage information of all RDDs persisted in the given SparkContext. */ - def rddInfoFromStorageStatus( - storageStatuses: Seq[StorageStatus], - sc: SparkContext): Array[RDDInfo] = { - val rddInfos = sc.persistentRdds.values.map { rdd => - val rddName = Option(rdd.name).getOrElse(rdd.id.toString) - val rddNumPartitions = rdd.partitions.size - val rddStorageLevel = rdd.getStorageLevel - val rddInfo = new RDDInfo(rdd.id, rddName, rddNumPartitions, rddStorageLevel) + /** Return a list of RDDInfo based on the RDDs cached in the given SparkContext. */ + def makeRddInfo(sc: SparkContext): Seq[RDDInfo] = { + sc.persistentRdds.values.toSeq.map { rdd => + val name = Option(rdd.name).getOrElse(rdd.id.toString) + val numPartitions = rdd.partitions.size + val storageLevel = rdd.getStorageLevel + val rddInfo = new RDDInfo(rdd.id, name, numPartitions, storageLevel) rddInfo - }.toArray - rddInfoFromStorageStatus(storageStatuses, rddInfos) - rddInfos + } } - /** Returns storage information of all RDDs in the given list. */ - def rddInfoFromStorageStatus( - storageStatuses: Seq[StorageStatus], - rddInfos: Seq[RDDInfo], - updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Unit = { - + /** Update the given list of RDDInfo with the given list of storage statuses. */ + def updateRddInfo(storageStatuses: Seq[StorageStatus], rddInfos: Seq[RDDInfo]): Unit = { // Mapping from a block ID -> its status val blockMap = mutable.Map(storageStatuses.flatMap(_.rddBlocks): _*) - // Record updated blocks, if any - updatedBlocks - .collect { case (id: RDDBlockId, status) => (id, status) } - .foreach { case (id, status) => blockMap(id) = status } - // Mapping from RDD ID -> an array of associated BlockStatuses val rddBlockMap = blockMap .groupBy { case (k, _) => k.rddId } @@ -165,22 +154,23 @@ private[spark] object StorageUtils { } } - /** Returns a mapping from BlockId to the locations of the associated block. */ - def blockLocationsFromStorageStatus( - storageStatuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { - // An ungrouped list of (blockId, location) pairs - val blockLocationPairs = storageStatuses.flatMap { storageStatus => - storageStatus.blocks.map { case (bid, _) => (bid, storageStatus.blockManagerId.hostPort) } + /** Return a mapping from block ID to the locations of the associated block. */ + def getBlockLocations(storageStatuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { + val blockLocations = new mutable.HashMap[BlockId, mutable.ListBuffer[String]] + storageStatuses.foreach { status => + status.blocks.foreach { case (bid, _) => + val location = status.blockManagerId.hostPort + blockLocations.getOrElseUpdate(bid, mutable.ListBuffer.empty) += location + } } - blockLocationPairs - .groupBy { case (blockId, _) => blockId } - .mapValues { rddLocations => rddLocations.map { case (_, location) => location } } + blockLocations } - /** Filters the given list of StorageStatus by the given RDD ID. */ - def filterStorageStatusByRDD( - storageStatuses: Seq[StorageStatus], - rddId: Int): Array[StorageStatus] = { + /** + * Return a filtered list of storage statuses in which the only blocks remaining are the ones + * that belong to given RDD. + */ + def filterByRDD(storageStatuses: Seq[StorageStatus], rddId: Int): Seq[StorageStatus] = { storageStatuses .filter(_.rddIds.contains(rddId)) .map { status => @@ -188,6 +178,6 @@ private[spark] object StorageUtils { status.blockManagerId, status.maxMem, status.rddBlocks(rddId).asInstanceOf[Map[BlockId, BlockStatus]]) - }.toArray + } } } 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 2155633b8096..c3d0aa3d364a 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 @@ -45,9 +45,9 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) // Block table - val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, rddId) + val filteredStorageStatusList = StorageUtils.filterByRDD(storageStatusList, rddId) val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).sortWith(_._1.name < _._1.name) - val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) + val blockLocations = StorageUtils.getBlockLocations(filteredStorageStatusList) val blocks = blockStatuses.map { case (blockId, status) => (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 73242518756b..f397ffcd604c 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -48,10 +48,9 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar /** Filter RDD info to include only those with cached partitions */ def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq - /** Update each RDD's info to reflect any updates to the RDD's storage status */ - private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty) { - val rddInfos = _rddInfoMap.values.toSeq - StorageUtils.rddInfoFromStorageStatus(storageStatusList, rddInfos, updatedBlocks) + /** Update each RDD's info to reflect any updates in the RDD's storage status */ + private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Unit = { + StorageUtils.updateRddInfo(storageStatusList, _rddInfoMap.values.toSeq) } /** 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 a265a081a4bf..93ba3f18daf3 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -179,10 +179,10 @@ class StorageSuite extends FunSuite { assert(status.diskUsedByRDD(3) === 200L) } - test("StorageUtils.rddInfoFromStorageStatus") { + test("StorageUtils.updateRddInfo") { val storageStatuses = stockStorageStatuses val rddInfos = stockRDDInfos - StorageUtils.rddInfoFromStorageStatus(storageStatuses, rddInfos) + StorageUtils.updateRddInfo(storageStatuses, rddInfos) assert(rddInfos(0).numCachedPartitions === 5) assert(rddInfos(0).memSize === 5L) assert(rddInfos(0).diskSize === 10L) @@ -191,54 +191,54 @@ class StorageSuite extends FunSuite { assert(rddInfos(1).diskSize === 6L) } - test("StorageUtils.rddInfoFromStorageStatus with updated blocks") { - val storageStatuses = stockStorageStatuses - val rddInfos = stockRDDInfos - - // Drop 3 blocks from RDD 0, and cache more of RDD 1 - val updatedBlocks1 = Seq( - (RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(0, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(0, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(1, 0), BlockStatus(memAndDisk, 100L, 100L, 0L)), - (RDDBlockId(1, 100), BlockStatus(memAndDisk, 100L, 100L, 0L)) - ) - StorageUtils.rddInfoFromStorageStatus(storageStatuses, rddInfos, updatedBlocks1) - assert(rddInfos(0).numCachedPartitions === 2) - assert(rddInfos(0).memSize === 2L) - assert(rddInfos(0).diskSize === 4L) - assert(rddInfos(1).numCachedPartitions === 4) - assert(rddInfos(1).memSize === 202L) - assert(rddInfos(1).diskSize === 204L) - - // Actually update storage statuses so we can chain the calls to rddInfoFromStorageStatus - updatedBlocks1.foreach { case (bid, bstatus) => - val statusWithBlock = storageStatuses.find(_.blocks.contains(bid)) - statusWithBlock match { - case Some(s) => s.updateBlock(bid, bstatus) - case None => storageStatuses(0).addBlock(bid, bstatus) // arbitrarily pick the first - } - } - - // Drop all of RDD 1 - val updatedBlocks2 = Seq( - (RDDBlockId(1, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(1, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(1, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(1, 100), BlockStatus(memAndDisk, 0L, 0L, 0L)) - ) - StorageUtils.rddInfoFromStorageStatus(storageStatuses, rddInfos, updatedBlocks2) - assert(rddInfos(0).numCachedPartitions === 2) - assert(rddInfos(0).memSize === 2L) - assert(rddInfos(0).diskSize === 4L) - assert(rddInfos(1).numCachedPartitions === 0) - assert(rddInfos(1).memSize === 0L) - assert(rddInfos(1).diskSize === 0L) - } +// test("StorageUtils.rddInfoFromStorageStatus with updated blocks") { +// val storageStatuses = stockStorageStatuses +// val rddInfos = stockRDDInfos +// +// // Drop 3 blocks from RDD 0, and cache more of RDD 1 +// val updatedBlocks1 = Seq( +// (RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), +// (RDDBlockId(0, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), +// (RDDBlockId(0, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), +// (RDDBlockId(1, 0), BlockStatus(memAndDisk, 100L, 100L, 0L)), +// (RDDBlockId(1, 100), BlockStatus(memAndDisk, 100L, 100L, 0L)) +// ) +// StorageUtils.rddInfoFromStorageStatus(storageStatuses, rddInfos, updatedBlocks1) +// assert(rddInfos(0).numCachedPartitions === 2) +// assert(rddInfos(0).memSize === 2L) +// assert(rddInfos(0).diskSize === 4L) +// assert(rddInfos(1).numCachedPartitions === 4) +// assert(rddInfos(1).memSize === 202L) +// assert(rddInfos(1).diskSize === 204L) +// +// // Actually update storage statuses so we can chain the calls to rddInfoFromStorageStatus +// updatedBlocks1.foreach { case (bid, bstatus) => +// val statusWithBlock = storageStatuses.find(_.blocks.contains(bid)) +// statusWithBlock match { +// case Some(s) => s.updateBlock(bid, bstatus) +// case None => storageStatuses(0).addBlock(bid, bstatus) // arbitrarily pick the first +// } +// } +// +// // Drop all of RDD 1 +// val updatedBlocks2 = Seq( +// (RDDBlockId(1, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), +// (RDDBlockId(1, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), +// (RDDBlockId(1, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), +// (RDDBlockId(1, 100), BlockStatus(memAndDisk, 0L, 0L, 0L)) +// ) +// StorageUtils.rddInfoFromStorageStatus(storageStatuses, rddInfos, updatedBlocks2) +// assert(rddInfos(0).numCachedPartitions === 2) +// assert(rddInfos(0).memSize === 2L) +// assert(rddInfos(0).diskSize === 4L) +// assert(rddInfos(1).numCachedPartitions === 0) +// assert(rddInfos(1).memSize === 0L) +// assert(rddInfos(1).diskSize === 0L) +// } - test("StorageUtils.blockLocationsFromStorageStatus") { + test("StorageUtils.getBlockLocations") { val storageStatuses = stockStorageStatuses - val blockLocations1 = StorageUtils.blockLocationsFromStorageStatus(storageStatuses) + val blockLocations1 = StorageUtils.getBlockLocations(storageStatuses) assert(blockLocations1.contains(RDDBlockId(0, 0))) assert(blockLocations1.contains(RDDBlockId(0, 1))) assert(blockLocations1.contains(RDDBlockId(0, 2))) @@ -261,7 +261,7 @@ class StorageSuite extends FunSuite { storageStatuses(0).addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) storageStatuses(0).addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) storageStatuses(2).addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) - val blockLocations2 = StorageUtils.blockLocationsFromStorageStatus(storageStatuses) + val blockLocations2 = StorageUtils.getBlockLocations(storageStatuses) assert(blockLocations2.contains(RDDBlockId(0, 0))) assert(blockLocations2.contains(RDDBlockId(0, 1))) assert(blockLocations2.contains(RDDBlockId(0, 2))) @@ -281,10 +281,10 @@ class StorageSuite extends FunSuite { assert(blockLocations2(RDDBlockId(1, 2)) === Seq("cat:3")) } - test("StorageUtils.filterStorageStatusByRDD") { + test("StorageUtils.filterByRDD") { val storageStatuses = stockStorageStatuses - val filteredStorageStatuses0 = StorageUtils.filterStorageStatusByRDD(storageStatuses, 0) - val filteredStorageStatuses1 = StorageUtils.filterStorageStatusByRDD(storageStatuses, 1) + val filteredStorageStatuses0 = StorageUtils.filterByRDD(storageStatuses, 0) + val filteredStorageStatuses1 = StorageUtils.filterByRDD(storageStatuses, 1) assert(filteredStorageStatuses0.size === 3) assert(filteredStorageStatuses1.size === 2) } From 8e91921983fc385896d9946303debd9e77652d6c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 30 Jul 2014 20:21:31 -0700 Subject: [PATCH 04/16] Iterate through a filtered set of blocks when updating RDDInfo This particular commit is the whole point of this PR. In the existing code we unconditionally iterate through all blocks in all block managers whenever we want to update an RDDInfo. Now, we filter out only the blocks of interest to us in advance, so we don't end up constructing a huge map and doing a groupBy on it. --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../apache/spark/storage/StorageUtils.scala | 57 ++++++------ .../apache/spark/ui/storage/StorageTab.scala | 4 +- .../apache/spark/storage/StorageSuite.scala | 90 +++++++++---------- 4 files changed, 78 insertions(+), 75 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8b1916d7badf..564020962a8a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -841,7 +841,7 @@ class SparkContext(config: SparkConf) extends Logging { @DeveloperApi def getRDDStorageInfo: Array[RDDInfo] = { val rddInfos = StorageUtils.makeRddInfo(this) - StorageUtils.updateRddInfo(getExecutorStorageStatus, rddInfos) + StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus) rddInfos.toArray } 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 06727637d22f..3b3fff0c0646 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -121,36 +121,39 @@ private[spark] object StorageUtils { } } - /** Update the given list of RDDInfo with the given list of storage statuses. */ - def updateRddInfo(storageStatuses: Seq[StorageStatus], rddInfos: Seq[RDDInfo]): Unit = { - // Mapping from a block ID -> its status - val blockMap = mutable.Map(storageStatuses.flatMap(_.rddBlocks): _*) - - // Mapping from RDD ID -> an array of associated BlockStatuses - val rddBlockMap = blockMap - .groupBy { case (k, _) => k.rddId } - .mapValues(_.values.toArray) - - // Mapping from RDD ID -> the associated RDDInfo (with potentially outdated storage information) - val rddInfoMap = rddInfos.map { info => (info.id, info) }.toMap - - rddBlockMap.foreach { case (rddId, blocks) => - // Add up memory, disk and Tachyon sizes - val persistedBlocks = - blocks.filter { status => status.memSize + status.diskSize + status.tachyonSize > 0 } - val _storageLevel = - if (persistedBlocks.length > 0) persistedBlocks(0).storageLevel else StorageLevel.NONE + /** + * Update the given list of RDDInfo with the given list of storage statuses. + * This method overwrites the old values stored in the RDDInfo's. + */ + def updateRddInfo( + rddInfos: Seq[RDDInfo], + storageStatuses: Seq[StorageStatus], + updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Unit = { + rddInfos.foreach { rddInfo => + val rddId = rddInfo.id + + // Collect all block statuses that belong to the given RDD + val newBlocks = updatedBlocks + .collect { case (bid: RDDBlockId, bstatus) => (bid, bstatus) } + .filter { case (bid, _) => bid.rddId == rddId } + val newBlockIds = newBlocks.map { case (bid, _) => bid }.toSet + val oldBlocks = storageStatuses + .filter(_.rddIds.contains(rddId)) + .flatMap(_.rddBlocks(rddId)) + .filter { case (bid, _) => !newBlockIds.contains(bid) } // avoid duplicates + val blocks = (oldBlocks ++ newBlocks).map { case (_, bstatus) => bstatus } + val persistedBlocks = blocks.filter { s => s.memSize + s.diskSize + s.tachyonSize > 0 } + + // Assume all blocks belonging to the same RDD have the same storage level + val storageLevel = blocks.headOption.map(_.storageLevel).getOrElse(StorageLevel.NONE) val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) val tachyonSize = persistedBlocks.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) - rddInfoMap.get(rddId).map { rddInfo => - rddInfo.storageLevel = _storageLevel - rddInfo.numCachedPartitions = persistedBlocks.length - rddInfo.memSize = memSize - rddInfo.diskSize = diskSize - rddInfo.tachyonSize = tachyonSize - rddInfo - } + rddInfo.storageLevel = storageLevel + rddInfo.numCachedPartitions = persistedBlocks.length + rddInfo.memSize = memSize + rddInfo.diskSize = diskSize + rddInfo.tachyonSize = tachyonSize } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index f397ffcd604c..854430d33724 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -49,8 +49,8 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq /** Update each RDD's info to reflect any updates in the RDD's storage status */ - private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Unit = { - StorageUtils.updateRddInfo(storageStatusList, _rddInfoMap.values.toSeq) + private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { + StorageUtils.updateRddInfo(_rddInfoMap.values.toSeq, storageStatusList, updatedBlocks) } /** 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 93ba3f18daf3..fce45462655f 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -182,7 +182,7 @@ class StorageSuite extends FunSuite { test("StorageUtils.updateRddInfo") { val storageStatuses = stockStorageStatuses val rddInfos = stockRDDInfos - StorageUtils.updateRddInfo(storageStatuses, rddInfos) + StorageUtils.updateRddInfo(rddInfos, storageStatuses) assert(rddInfos(0).numCachedPartitions === 5) assert(rddInfos(0).memSize === 5L) assert(rddInfos(0).diskSize === 10L) @@ -191,50 +191,50 @@ class StorageSuite extends FunSuite { assert(rddInfos(1).diskSize === 6L) } -// test("StorageUtils.rddInfoFromStorageStatus with updated blocks") { -// val storageStatuses = stockStorageStatuses -// val rddInfos = stockRDDInfos -// -// // Drop 3 blocks from RDD 0, and cache more of RDD 1 -// val updatedBlocks1 = Seq( -// (RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), -// (RDDBlockId(0, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), -// (RDDBlockId(0, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), -// (RDDBlockId(1, 0), BlockStatus(memAndDisk, 100L, 100L, 0L)), -// (RDDBlockId(1, 100), BlockStatus(memAndDisk, 100L, 100L, 0L)) -// ) -// StorageUtils.rddInfoFromStorageStatus(storageStatuses, rddInfos, updatedBlocks1) -// assert(rddInfos(0).numCachedPartitions === 2) -// assert(rddInfos(0).memSize === 2L) -// assert(rddInfos(0).diskSize === 4L) -// assert(rddInfos(1).numCachedPartitions === 4) -// assert(rddInfos(1).memSize === 202L) -// assert(rddInfos(1).diskSize === 204L) -// -// // Actually update storage statuses so we can chain the calls to rddInfoFromStorageStatus -// updatedBlocks1.foreach { case (bid, bstatus) => -// val statusWithBlock = storageStatuses.find(_.blocks.contains(bid)) -// statusWithBlock match { -// case Some(s) => s.updateBlock(bid, bstatus) -// case None => storageStatuses(0).addBlock(bid, bstatus) // arbitrarily pick the first -// } -// } -// -// // Drop all of RDD 1 -// val updatedBlocks2 = Seq( -// (RDDBlockId(1, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), -// (RDDBlockId(1, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), -// (RDDBlockId(1, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), -// (RDDBlockId(1, 100), BlockStatus(memAndDisk, 0L, 0L, 0L)) -// ) -// StorageUtils.rddInfoFromStorageStatus(storageStatuses, rddInfos, updatedBlocks2) -// assert(rddInfos(0).numCachedPartitions === 2) -// assert(rddInfos(0).memSize === 2L) -// assert(rddInfos(0).diskSize === 4L) -// assert(rddInfos(1).numCachedPartitions === 0) -// assert(rddInfos(1).memSize === 0L) -// assert(rddInfos(1).diskSize === 0L) -// } + test("StorageUtils.updateRddInfo with updated blocks") { + val storageStatuses = stockStorageStatuses + val rddInfos = stockRDDInfos + + // Drop 3 blocks from RDD 0, and cache more of RDD 1 + val updatedBlocks1 = Seq( + (RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(0, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(0, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(1, 0), BlockStatus(memAndDisk, 100L, 100L, 0L)), + (RDDBlockId(1, 100), BlockStatus(memAndDisk, 100L, 100L, 0L)) + ) + StorageUtils.updateRddInfo(rddInfos, storageStatuses, updatedBlocks1) + assert(rddInfos(0).numCachedPartitions === 2) + assert(rddInfos(0).memSize === 2L) + assert(rddInfos(0).diskSize === 4L) + assert(rddInfos(1).numCachedPartitions === 4) + assert(rddInfos(1).memSize === 202L) + assert(rddInfos(1).diskSize === 204L) + + // Actually update storage statuses so we can chain the calls to rddInfoFromStorageStatus + updatedBlocks1.foreach { case (bid, bstatus) => + val statusWithBlock = storageStatuses.find(_.blocks.contains(bid)) + statusWithBlock match { + case Some(s) => s.updateBlock(bid, bstatus) + case None => storageStatuses(0).addBlock(bid, bstatus) // arbitrarily pick the first + } + } + + // Drop all of RDD 1 + val updatedBlocks2 = Seq( + (RDDBlockId(1, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(1, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(1, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), + (RDDBlockId(1, 100), BlockStatus(memAndDisk, 0L, 0L, 0L)) + ) + StorageUtils.updateRddInfo(rddInfos, storageStatuses, updatedBlocks2) + assert(rddInfos(0).numCachedPartitions === 2) + assert(rddInfos(0).memSize === 2L) + assert(rddInfos(0).diskSize === 4L) + assert(rddInfos(1).numCachedPartitions === 0) + assert(rddInfos(1).memSize === 0L) + assert(rddInfos(1).diskSize === 0L) + } test("StorageUtils.getBlockLocations") { val storageStatuses = stockStorageStatuses From b12fcd7e2e0adb769cd5d1c6c37dee77a986c234 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 31 Jul 2014 12:14:46 -0700 Subject: [PATCH 05/16] Fix tests + simplify sc.getRDDStorageInfo This adds more beef to the test that was failing. --- .../scala/org/apache/spark/SparkContext.scala | 6 ++--- .../storage/BlockManagerMasterActor.scala | 4 +++- .../org/apache/spark/storage/RDDInfo.scala | 2 ++ .../apache/spark/storage/StorageUtils.scala | 13 +---------- .../apache/spark/SparkContextInfoSuite.scala | 22 +++++++++++++------ 5 files changed, 24 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3403d0d380db..17e9ef902621 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -47,7 +47,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend -import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} +import org.apache.spark.storage._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} @@ -840,9 +840,9 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi def getRDDStorageInfo: Array[RDDInfo] = { - val rddInfos = StorageUtils.makeRddInfo(this) + val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus) - rddInfos.toArray + rddInfos.filter(_.isCached) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index f76ab323d3ef..9b13f32fff78 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -420,7 +420,9 @@ case class BlockStatus( storageLevel: StorageLevel, memSize: Long, diskSize: Long, - tachyonSize: Long) + tachyonSize: Long) { + def isCached: Boolean = memSize + diskSize + tachyonSize > 0 +} private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 5a72e216872a..120c327a7e58 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -34,6 +34,8 @@ class RDDInfo( var diskSize = 0L var tachyonSize = 0L + def isCached: Boolean = (memSize + diskSize + tachyonSize > 0) && numCachedPartitions > 0 + override def toString = { import Utils.bytesToString ("RDD \"%s\" (%d) StorageLevel: %s; CachedPartitions: %d; TotalPartitions: %d; " + 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 3b3fff0c0646..446a3887630d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -110,17 +110,6 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** Helper methods for storage-related objects. */ private[spark] object StorageUtils { - /** Return a list of RDDInfo based on the RDDs cached in the given SparkContext. */ - def makeRddInfo(sc: SparkContext): Seq[RDDInfo] = { - sc.persistentRdds.values.toSeq.map { rdd => - val name = Option(rdd.name).getOrElse(rdd.id.toString) - val numPartitions = rdd.partitions.size - val storageLevel = rdd.getStorageLevel - val rddInfo = new RDDInfo(rdd.id, name, numPartitions, storageLevel) - rddInfo - } - } - /** * Update the given list of RDDInfo with the given list of storage statuses. * This method overwrites the old values stored in the RDDInfo's. @@ -142,7 +131,7 @@ private[spark] object StorageUtils { .flatMap(_.rddBlocks(rddId)) .filter { case (bid, _) => !newBlockIds.contains(bid) } // avoid duplicates val blocks = (oldBlocks ++ newBlocks).map { case (_, bstatus) => bstatus } - val persistedBlocks = blocks.filter { s => s.memSize + s.diskSize + s.tachyonSize > 0 } + val persistedBlocks = blocks.filter(_.isCached) // Assume all blocks belonging to the same RDD have the same storage level val storageLevel = blocks.headOption.map(_.storageLevel).getOrElse(StorageLevel.NONE) diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index fb18c3ebfe46..e6ab538d77bc 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark import org.scalatest.{Assertions, FunSuite} +import org.apache.spark.storage.StorageLevel class SparkContextInfoSuite extends FunSuite with LocalSparkContext { test("getPersistentRDDs only returns RDDs that are marked as cached") { @@ -35,26 +36,33 @@ class SparkContextInfoSuite extends FunSuite with LocalSparkContext { test("getPersistentRDDs returns an immutable map") { sc = new SparkContext("local", "test") val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() - val myRdds = sc.getPersistentRDDs assert(myRdds.size === 1) - assert(myRdds.values.head === rdd1) + assert(myRdds(0) === rdd1) + assert(myRdds(0).getStorageLevel === StorageLevel.MEMORY_ONLY) + // myRdds2 should have 2 RDDs, but myRdds should not change val rdd2 = sc.makeRDD(Array(5, 6, 7, 8), 1).cache() - - // getPersistentRDDs should have 2 RDDs, but myRdds should not change - assert(sc.getPersistentRDDs.size === 2) + val myRdds2 = sc.getPersistentRDDs + assert(myRdds2.size === 2) + assert(myRdds2(0) === rdd1) + assert(myRdds2(1) === rdd2) + assert(myRdds2(0).getStorageLevel === StorageLevel.MEMORY_ONLY) + assert(myRdds2(1).getStorageLevel === StorageLevel.MEMORY_ONLY) assert(myRdds.size === 1) + assert(myRdds(0) === rdd1) + assert(myRdds(0).getStorageLevel === StorageLevel.MEMORY_ONLY) } test("getRDDStorageInfo only reports on RDDs that actually persist data") { sc = new SparkContext("local", "test") val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() - assert(sc.getRDDStorageInfo.size === 0) - rdd.collect() assert(sc.getRDDStorageInfo.size === 1) + assert(sc.getRDDStorageInfo.head.isCached) + assert(sc.getRDDStorageInfo.head.memSize > 0) + assert(sc.getRDDStorageInfo.head.storageLevel === StorageLevel.MEMORY_ONLY) } test("call sites report correct locations") { From 6a7b7c0f88abe976496ca54b4e01a6018c13a5e6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 31 Jul 2014 14:07:02 -0700 Subject: [PATCH 06/16] Avoid chained operations on TraversableLike Chaining filter / map / flatMaps etc. can be expensive if the underlying collection is huge, because each of these operations creates a copy of the collection and applies a function to it. In terms of actual code, updateRddInfo is rewritten to iterate through each collection only once. We used to do a filter, then a flatMap, then another filter on all existing blocks. Now, we move the filter into the flatMap, and replace it with filterKeys, which does not copy the underlying map. Unfortunately this sacrifices code readability a little bit, but the performance gain should be worth it. --- .../apache/spark/storage/StorageUtils.scala | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) 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 446a3887630d..537779b7f22f 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -118,18 +118,23 @@ private[spark] object StorageUtils { rddInfos: Seq[RDDInfo], storageStatuses: Seq[StorageStatus], updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Unit = { + rddInfos.foreach { rddInfo => val rddId = rddInfo.id // Collect all block statuses that belong to the given RDD - val newBlocks = updatedBlocks - .collect { case (bid: RDDBlockId, bstatus) => (bid, bstatus) } - .filter { case (bid, _) => bid.rddId == rddId } + val newBlocks = updatedBlocks.filter { case (b, _) => + b.asRDDId.filter(_.rddId == rddId).isDefined + } val newBlockIds = newBlocks.map { case (bid, _) => bid }.toSet - val oldBlocks = storageStatuses - .filter(_.rddIds.contains(rddId)) - .flatMap(_.rddBlocks(rddId)) - .filter { case (bid, _) => !newBlockIds.contains(bid) } // avoid duplicates + val oldBlocks = storageStatuses.flatMap { s => + if (s.rddIds.contains(rddId)) { + // If the block is being updated, leave it out here in favor of the new status + s.rddBlocks(rddId).filterKeys { bid => !newBlockIds.contains(bid) } + } else { + Seq.empty + } + } val blocks = (oldBlocks ++ newBlocks).map { case (_, bstatus) => bstatus } val persistedBlocks = blocks.filter(_.isCached) From b66b6b0ab3077ae2e1fcce331b8efe42fb28c8c5 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 31 Jul 2014 19:21:58 -0700 Subject: [PATCH 07/16] Use more efficient underlying data structures for blocks Previously we were still linearly traversing all the blocks held by each storage status. Now we index by the RDD ID and return only the blocks of interest to us. --- .../spark/storage/StorageStatusListener.scala | 2 +- .../apache/spark/storage/StorageUtils.scala | 144 +++++++++++------- .../apache/spark/ui/exec/ExecutorsPage.scala | 2 +- .../org/apache/spark/ui/storage/RDDPage.scala | 13 +- .../storage/StorageStatusListenerSuite.scala | 72 ++++----- .../apache/spark/storage/StorageSuite.scala | 134 ++++++++-------- 6 files changed, 192 insertions(+), 175 deletions(-) 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 7b75afddef68..3966c33e3fb9 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -50,7 +50,7 @@ class StorageStatusListener extends SparkListener { /** Update storage status list to reflect the removal of an RDD from the cache */ private def updateStorageStatus(unpersistedRDDId: Int) { storageStatusList.foreach { storageStatus => - storageStatus.rddBlocks(unpersistedRDDId).foreach { case (blockId, _) => + storageStatus.rddBlocksById(unpersistedRDDId).foreach { case (blockId, _) => storageStatus.removeBlock(blockId) } } 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 537779b7f22f..e1f2ae2f3457 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -20,7 +20,6 @@ package org.apache.spark.storage import scala.collection.Map import scala.collection.mutable -import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi /** @@ -31,9 +30,24 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { - // This should not be mutated directly, but through the add/update/removeBlock methods - private val _blocks = new mutable.HashMap[BlockId, BlockStatus] - private val _rddIds = new mutable.HashSet[Int] + /** + * Internal representation of the blocks stored in this block manager. + * + * Common consumption patterns of these blocks include + * (1) selecting all blocks, + * (2) selecting only RDD blocks or, + * (3) selecting only the blocks that belong to a specific RDD + * + * If we are only interested in a fraction of the blocks, as in (2) and (3), we should avoid + * linearly scanning through all the blocks, which could be expensive if there are thousands + * of blocks on each block manager. We achieve this by storing RDD blocks and non-RDD blocks + * separately. In particular, RDD blocks are stored in a map indexed by RDD IDs, so we can + * filter out the blocks of interest quickly. + * + * These collections should only be mutated through the add/update/removeBlock methods. + */ + private val _rddBlocks = new mutable.HashMap[Int, mutable.Map[BlockId, BlockStatus]] + private val _nonRddBlocks = new mutable.HashMap[BlockId, BlockStatus] /** * Instantiate a StorageStatus with the given initial blocks. This essentially makes a copy of @@ -44,67 +58,94 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { initialBlocks.foreach { case (blockId, blockStatus) => addBlock(blockId, blockStatus) } } - /** Return the blocks stored in this block manager as a mapping from ID to status. */ - def blocks: Map[BlockId, BlockStatus] = _blocks + /** Return the blocks stored in this block manager. */ + def blocks: Seq[(BlockId, BlockStatus)] = { + _nonRddBlocks.toSeq ++ rddBlocks.toSeq + } + + /** Return the RDD blocks stored in this block manager. */ + def rddBlocks: Seq[(BlockId, BlockStatus)] = { + _rddBlocks.flatMap { case (_, blocks) => blocks }.toSeq + } + + /** Return the blocks that belong to the given RDD stored in this block manager. */ + def rddBlocksById(rddId: Int): Seq[(BlockId, BlockStatus)] = { + _rddBlocks.get(rddId).map(_.toSeq).getOrElse(Seq.empty) + } - /** Add the given block, keeping track of the RDD ID if this is an RDD block. */ + /** Add the given block to this storage status. */ def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { blockId match { - case RDDBlockId(rddId, _) => _rddIds.add(rddId) + case RDDBlockId(rddId, _) => + _rddBlocks.getOrElseUpdate(rddId, new mutable.HashMap)(blockId) = blockStatus case _ => + _nonRddBlocks(blockId) = blockStatus } - _blocks(blockId) = blockStatus } - /** Update the given block, keeping track of the RDD ID if this is an RDD block. */ + /** Update the given block in this storage status. If it doesn't already exist, add it. */ def updateBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = addBlock(blockId, blockStatus) - /** Remove the given block, keeping track of the RDD ID if this is an RDD block. */ + /** Remove the given block from this storage status. */ def removeBlock(blockId: BlockId): Option[BlockStatus] = { - val removed = _blocks.remove(blockId) blockId match { case RDDBlockId(rddId, _) => - if (rddBlocks(rddId).isEmpty) { - _rddIds.remove(rddId) + if (_rddBlocks.contains(rddId)) { + val removed = _rddBlocks(rddId).remove(blockId) + // If the given RDD has no more blocks left, remove the RDD + if (_rddBlocks(rddId).isEmpty) { + _rddBlocks.remove(rddId) + } + removed + } else { + None } case _ => + _nonRddBlocks.remove(blockId) } - removed } - /** Return the IDs of the RDDs which have blocks stored in this block manager. */ - def rddIds: Seq[Int] = _rddIds.toSeq - - /** Return the RDD blocks stored in this block manager as a mapping from ID to status. */ - def rddBlocks: Map[RDDBlockId, BlockStatus] = - blocks.filterKeys(_.isInstanceOf[RDDBlockId]).asInstanceOf[Map[RDDBlockId, BlockStatus]] + /** + * Return whether the given block is stored in this block manager in O(1) time. + * Note that the alternative of doing this through `blocks` is O(blocks), which is much slower. + */ + def containsBlock(blockId: BlockId): Boolean = { + blockId match { + case RDDBlockId(rddId, _) => + _rddBlocks.get(rddId).filter(_.contains(blockId)).isDefined + case _ => + _nonRddBlocks.contains(blockId) + } + } /** - * Return the RDD blocks with the given RDD ID stored in this block manager as a mapping - * from ID to status. + * Return the number of blocks in O(R) time, where R is the number of distinct RDD IDs. + * Note that the alternative of doing this through `blocks` is O(blocks), which is much slower. */ - def rddBlocks(rddId: Int): Map[RDDBlockId, BlockStatus] = rddBlocks.filterKeys(_.rddId == rddId) + def numBlocks: Int = { + _nonRddBlocks.size + _rddBlocks.values.map(_.size).reduceOption(_ + _).getOrElse(0) + } /** Return the memory used by this block manager. */ - def memUsed: Long = memUsed(blocks.values) + def memUsed: Long = memUsed(blocks) /** Return the memory used by the given RDD in this block manager. */ - def memUsedByRDD(rddId: Int): Long = memUsed(rddBlocks(rddId).values) + def memUsedByRDD(rddId: Int): Long = memUsed(rddBlocksById(rddId)) /** Return the memory remaining in this block manager. */ def memRemaining: Long = maxMem - memUsed /** Return the disk space used by this block manager. */ - def diskUsed: Long = diskUsed(blocks.values) + def diskUsed: Long = diskUsed(blocks) /** Return the disk space used by the given RDD in this block manager. */ - def diskUsedByRDD(rddId: Int): Long = diskUsed(rddBlocks(rddId).values) + def diskUsedByRDD(rddId: Int): Long = diskUsed(rddBlocksById(rddId)) // Helper methods for computing memory and disk usages - private def memUsed(statuses: Iterable[BlockStatus]): Long = - statuses.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - private def diskUsed(statuses: Iterable[BlockStatus]): Long = - statuses.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + private def memUsed(_blocks: Seq[(BlockId, BlockStatus)]): Long = + _blocks.map { case (_, s) => s.memSize }.reduceOption(_ + _).getOrElse(0L) + private def diskUsed(_blocks: Seq[(BlockId, BlockStatus)]): Long = + _blocks.map { case (_, s) => s.diskSize }.reduceOption(_ + _).getOrElse(0L) } /** Helper methods for storage-related objects. */ @@ -123,18 +164,13 @@ private[spark] object StorageUtils { val rddId = rddInfo.id // Collect all block statuses that belong to the given RDD - val newBlocks = updatedBlocks.filter { case (b, _) => - b.asRDDId.filter(_.rddId == rddId).isDefined + val newBlocks = updatedBlocks.filter { case (bid, _) => + bid.asRDDId.filter(_.rddId == rddId).isDefined } val newBlockIds = newBlocks.map { case (bid, _) => bid }.toSet - val oldBlocks = storageStatuses.flatMap { s => - if (s.rddIds.contains(rddId)) { - // If the block is being updated, leave it out here in favor of the new status - s.rddBlocks(rddId).filterKeys { bid => !newBlockIds.contains(bid) } - } else { - Seq.empty - } - } + val oldBlocks = storageStatuses + .flatMap(_.rddBlocksById(rddId)) + .filter { case (bid, _) => !newBlockIds.contains(bid) } // avoid double counting val blocks = (oldBlocks ++ newBlocks).map { case (_, bstatus) => bstatus } val persistedBlocks = blocks.filter(_.isCached) @@ -151,11 +187,15 @@ private[spark] object StorageUtils { } } - /** Return a mapping from block ID to the locations of the associated block. */ - def getBlockLocations(storageStatuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { + /** + * Return mapping from block ID to its locations for each block that belongs to the given RDD. + */ + def getRDDBlockLocations( + storageStatuses: Seq[StorageStatus], + rddId: Int): Map[BlockId, Seq[String]] = { val blockLocations = new mutable.HashMap[BlockId, mutable.ListBuffer[String]] storageStatuses.foreach { status => - status.blocks.foreach { case (bid, _) => + status.rddBlocksById(rddId).foreach { case (bid, _) => val location = status.blockManagerId.hostPort blockLocations.getOrElseUpdate(bid, mutable.ListBuffer.empty) += location } @@ -163,18 +203,4 @@ private[spark] object StorageUtils { blockLocations } - /** - * Return a filtered list of storage statuses in which the only blocks remaining are the ones - * that belong to given RDD. - */ - def filterByRDD(storageStatuses: Seq[StorageStatus], rddId: Int): Seq[StorageStatus] = { - storageStatuses - .filter(_.rddIds.contains(rddId)) - .map { status => - new StorageStatus( - status.blockManagerId, - status.maxMem, - status.rddBlocks(rddId).asInstanceOf[Map[BlockId, BlockStatus]]) - } - } } 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 46d4d68efbf2..cc8ef9f1b612 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 @@ -145,7 +145,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort - val rddBlocks = status.blocks.size + val rddBlocks = status.numBlocks val memUsed = status.memUsed val maxMem = status.maxMem 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 c3d0aa3d364a..2ae835e66363 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 @@ -45,12 +45,13 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) // Block table - val filteredStorageStatusList = StorageUtils.filterByRDD(storageStatusList, rddId) - val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).sortWith(_._1.name < _._1.name) - val blockLocations = StorageUtils.getBlockLocations(filteredStorageStatusList) - val blocks = blockStatuses.map { case (blockId, status) => - (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) - } + val blockLocations = StorageUtils.getRDDBlockLocations(storageStatusList, rddId) + val blocks = storageStatusList + .flatMap(_.rddBlocksById(rddId)) + .sortWith(_._1.name < _._1.name) + .map { case (blockId, status) => + (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) + } val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks) val content = diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 2179c6dd3302..51fb646a3cb6 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -41,13 +41,13 @@ class StorageStatusListenerSuite extends FunSuite { assert(listener.executorIdToStorageStatus.get("big").isDefined) assert(listener.executorIdToStorageStatus("big").blockManagerId === bm1) assert(listener.executorIdToStorageStatus("big").maxMem === 1000L) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) assert(listener.executorIdToStorageStatus.size === 2) assert(listener.executorIdToStorageStatus.get("fat").isDefined) assert(listener.executorIdToStorageStatus("fat").blockManagerId === bm2) assert(listener.executorIdToStorageStatus("fat").maxMem === 2000L) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) // Block manager remove listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(bm1)) @@ -67,14 +67,14 @@ class StorageStatusListenerSuite extends FunSuite { val taskMetrics = new TaskMetrics // Task end with no updated blocks - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics)) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics)) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) } test("task end with updated blocks") { @@ -90,20 +90,20 @@ class StorageStatusListenerSuite extends FunSuite { taskMetrics2.updatedBlocks = Some(Seq(block3)) // Task end with new blocks - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 2) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 0) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 2) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 2) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 1) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.contains(RDDBlockId(4, 0))) + assert(listener.executorIdToStorageStatus("big").numBlocks === 2) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) // Task end with dropped blocks val droppedBlock1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) @@ -112,17 +112,17 @@ class StorageStatusListenerSuite extends FunSuite { taskMetrics1.updatedBlocks = Some(Seq(droppedBlock1, droppedBlock3)) taskMetrics2.updatedBlocks = Some(Seq(droppedBlock2, droppedBlock3)) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 1) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 1) - assert(!listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.contains(RDDBlockId(4, 0))) + assert(listener.executorIdToStorageStatus("big").numBlocks === 1) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) + assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 1) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 0) - assert(!listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 1) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) + assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) } test("unpersist RDD") { @@ -137,16 +137,16 @@ class StorageStatusListenerSuite extends FunSuite { taskMetrics2.updatedBlocks = Some(Seq(block3)) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics2)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 3) + assert(listener.executorIdToStorageStatus("big").numBlocks === 3) // Unpersist RDD listener.onUnpersistRDD(SparkListenerUnpersistRDD(9090)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 3) + assert(listener.executorIdToStorageStatus("big").numBlocks === 3) listener.onUnpersistRDD(SparkListenerUnpersistRDD(4)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 2) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("big").numBlocks === 2) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) listener.onUnpersistRDD(SparkListenerUnpersistRDD(1)) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 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 fce45462655f..50924e4ce9ee 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -48,11 +48,10 @@ class StorageSuite extends FunSuite { Seq(info0, info1) } - test("storage status add/remove blocks") { + test("storage status add/remove non-RDD blocks") { val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) assert(status.blocks.isEmpty) assert(status.rddBlocks.isEmpty) - assert(status.rddIds.isEmpty) assert(status.memUsed === 0) assert(status.memRemaining === 1000L) assert(status.diskUsed === 0) @@ -63,7 +62,6 @@ class StorageSuite extends FunSuite { status.addBlock(TestBlockId("faa"), BlockStatus(memAndDisk, 10L, 20L, 0L)) assert(status.blocks.size === 3) assert(status.rddBlocks.isEmpty) - assert(status.rddIds.isEmpty) assert(status.memUsed === 30L) assert(status.memRemaining === 970L) assert(status.diskUsed === 60L) @@ -73,7 +71,6 @@ class StorageSuite extends FunSuite { status.updateBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 100L, 20L, 0L)) assert(status.blocks.size === 3) assert(status.rddBlocks.isEmpty) - assert(status.rddIds.isEmpty) assert(status.memUsed === 160L) assert(status.memRemaining === 840L) assert(status.diskUsed === 140L) @@ -83,7 +80,6 @@ class StorageSuite extends FunSuite { status.removeBlock(TestBlockId("faa")) assert(status.blocks.size === 1) assert(status.rddBlocks.isEmpty) - assert(status.rddIds.isEmpty) assert(status.memUsed === 100L) assert(status.memRemaining === 900L) assert(status.diskUsed === 20L) @@ -93,7 +89,6 @@ class StorageSuite extends FunSuite { val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) assert(status.blocks.isEmpty) assert(status.rddBlocks.isEmpty) - assert(status.rddIds.isEmpty) // Add a few blocks status.addBlock(TestBlockId("DANGER"), BlockStatus(memAndDisk, 10L, 20L, 0L)) @@ -105,16 +100,15 @@ class StorageSuite extends FunSuite { status.addBlock(RDDBlockId(2, 4), BlockStatus(memAndDisk, 10L, 40L, 0L)) assert(status.blocks.size === 7) assert(status.rddBlocks.size === 5) - assert(status.rddIds.toSet === Seq(0, 1, 2).toSet) - assert(status.rddBlocks(0).size === 1) - assert(status.rddBlocks(0).head._1 === RDDBlockId(0, 0)) - assert(status.rddBlocks(0).head._2 === BlockStatus(memAndDisk, 10L, 20L, 0L)) - assert(status.rddBlocks(1).size === 1) - assert(status.rddBlocks(1).head._1 === RDDBlockId(1, 1)) - assert(status.rddBlocks(1).head._2 === BlockStatus(memAndDisk, 100L, 200L, 0L)) - assert(status.rddBlocks(2).size === 3) - assert(status.rddBlocks(2).head._1 === RDDBlockId(2, 2)) - assert(status.rddBlocks(2).head._2 === BlockStatus(memAndDisk, 10L, 20L, 0L)) + assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(0).head._1 === RDDBlockId(0, 0)) + assert(status.rddBlocksById(0).head._2 === BlockStatus(memAndDisk, 10L, 20L, 0L)) + assert(status.rddBlocksById(1).size === 1) + assert(status.rddBlocksById(1).head._1 === RDDBlockId(1, 1)) + assert(status.rddBlocksById(1).head._2 === BlockStatus(memAndDisk, 100L, 200L, 0L)) + assert(status.rddBlocksById(2).size === 3) + assert(status.rddBlocksById(2).head._1 === RDDBlockId(2, 2)) + assert(status.rddBlocksById(2).head._2 === BlockStatus(memAndDisk, 10L, 20L, 0L)) assert(status.memUsedByRDD(0) === 10L) assert(status.memUsedByRDD(1) === 100L) assert(status.memUsedByRDD(2) === 30L) @@ -126,12 +120,14 @@ class StorageSuite extends FunSuite { status.addBlock(TestBlockId("DANGER"), BlockStatus(memAndDisk, 5000L, 0L, 0L)) status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)) status.updateBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 0L, 1000L, 0L)) - assert(status.rddBlocks(0).size === 1) - assert(status.rddBlocks(0).head._1 === RDDBlockId(0, 0)) - assert(status.rddBlocks(0).head._2 === BlockStatus(memAndDisk, 0L, 0L, 0L)) - assert(status.rddBlocks(2).size === 3) - assert(status.rddBlocks(2).head._1 === RDDBlockId(2, 2)) - assert(status.rddBlocks(2).head._2 === BlockStatus(memAndDisk, 0L, 1000L, 0L)) + assert(status.blocks.size === 7) + assert(status.rddBlocks.size === 5) + assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(0).head._1 === RDDBlockId(0, 0)) + assert(status.rddBlocksById(0).head._2 === BlockStatus(memAndDisk, 0L, 0L, 0L)) + assert(status.rddBlocksById(2).size === 3) + assert(status.rddBlocksById(2).head._1 === RDDBlockId(2, 2)) + assert(status.rddBlocksById(2).head._2 === BlockStatus(memAndDisk, 0L, 1000L, 0L)) assert(status.memUsedByRDD(0) === 0L) assert(status.memUsedByRDD(1) === 100L) assert(status.memUsedByRDD(2) === 20L) @@ -146,10 +142,9 @@ class StorageSuite extends FunSuite { status.removeBlock(RDDBlockId(2, 4)) assert(status.blocks.size === 3) assert(status.rddBlocks.size === 2) - assert(status.rddIds.toSet === Seq(0, 2).toSet) - assert(status.rddBlocks(0).size === 1) - assert(status.rddBlocks(1).size === 0) - assert(status.rddBlocks(2).size === 1) + assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(1).size === 0) + assert(status.rddBlocksById(2).size === 1) assert(status.memUsedByRDD(0) === 0L) assert(status.memUsedByRDD(1) === 0L) assert(status.memUsedByRDD(2) === 10L) @@ -162,13 +157,12 @@ class StorageSuite extends FunSuite { status.addBlock(RDDBlockId(3, 5), BlockStatus(memAndDisk, 10L, 200L, 0L)) assert(status.blocks.size === 5) assert(status.rddBlocks.size === 4) - assert(status.rddIds.toSet === Seq(0, 2, 3).toSet) - assert(status.rddBlocks(0).size === 1) - assert(status.rddBlocks(1).size === 0) - assert(status.rddBlocks(2).size === 2) - assert(status.rddBlocks(3).size === 1) - assert(status.rddBlocks(3).head._1 === RDDBlockId(3, 5)) - assert(status.rddBlocks(3).head._2 === BlockStatus(memAndDisk, 10L, 200L, 0L)) + assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(1).size === 0) + assert(status.rddBlocksById(2).size === 2) + assert(status.rddBlocksById(3).size === 1) + assert(status.rddBlocksById(3).head._1 === RDDBlockId(3, 5)) + assert(status.rddBlocksById(3).head._2 === BlockStatus(memAndDisk, 10L, 200L, 0L)) assert(status.memUsedByRDD(0) === 0L) assert(status.memUsedByRDD(1) === 0L) assert(status.memUsedByRDD(2) === 20L) @@ -213,7 +207,7 @@ class StorageSuite extends FunSuite { // Actually update storage statuses so we can chain the calls to rddInfoFromStorageStatus updatedBlocks1.foreach { case (bid, bstatus) => - val statusWithBlock = storageStatuses.find(_.blocks.contains(bid)) + val statusWithBlock = storageStatuses.find(_.containsBlock(bid)) statusWithBlock match { case Some(s) => s.updateBlock(bid, bstatus) case None => storageStatuses(0).addBlock(bid, bstatus) // arbitrarily pick the first @@ -238,55 +232,51 @@ class StorageSuite extends FunSuite { test("StorageUtils.getBlockLocations") { val storageStatuses = stockStorageStatuses - val blockLocations1 = StorageUtils.getBlockLocations(storageStatuses) - assert(blockLocations1.contains(RDDBlockId(0, 0))) - assert(blockLocations1.contains(RDDBlockId(0, 1))) - assert(blockLocations1.contains(RDDBlockId(0, 2))) - assert(blockLocations1.contains(RDDBlockId(0, 3))) - assert(blockLocations1.contains(RDDBlockId(0, 4))) + var blockLocations0 = StorageUtils.getRDDBlockLocations(storageStatuses, 0) + var blockLocations1 = StorageUtils.getRDDBlockLocations(storageStatuses, 1) + assert(blockLocations0.size === 5) + assert(blockLocations1.size === 3) + assert(blockLocations0.contains(RDDBlockId(0, 0))) + assert(blockLocations0.contains(RDDBlockId(0, 1))) + assert(blockLocations0.contains(RDDBlockId(0, 2))) + assert(blockLocations0.contains(RDDBlockId(0, 3))) + assert(blockLocations0.contains(RDDBlockId(0, 4))) assert(blockLocations1.contains(RDDBlockId(1, 0))) assert(blockLocations1.contains(RDDBlockId(1, 1))) assert(blockLocations1.contains(RDDBlockId(1, 2))) - assert(blockLocations1.size === 8) - assert(blockLocations1(RDDBlockId(0, 0)) === Seq("dog:1")) - assert(blockLocations1(RDDBlockId(0, 1)) === Seq("dog:1")) - assert(blockLocations1(RDDBlockId(0, 2)) === Seq("duck:2")) - assert(blockLocations1(RDDBlockId(0, 3)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 0)) === Seq("dog:1")) + assert(blockLocations0(RDDBlockId(0, 1)) === Seq("dog:1")) + assert(blockLocations0(RDDBlockId(0, 2)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 3)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 4)) === Seq("cat:3")) assert(blockLocations1(RDDBlockId(1, 0)) === Seq("duck:2")) assert(blockLocations1(RDDBlockId(1, 1)) === Seq("duck:2")) - assert(blockLocations1(RDDBlockId(0, 4)) === Seq("cat:3")) assert(blockLocations1(RDDBlockId(1, 2)) === Seq("cat:3")) // Multiple locations storageStatuses(0).addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) storageStatuses(0).addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) storageStatuses(2).addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) - val blockLocations2 = StorageUtils.getBlockLocations(storageStatuses) - assert(blockLocations2.contains(RDDBlockId(0, 0))) - assert(blockLocations2.contains(RDDBlockId(0, 1))) - assert(blockLocations2.contains(RDDBlockId(0, 2))) - assert(blockLocations2.contains(RDDBlockId(0, 3))) - assert(blockLocations2.contains(RDDBlockId(0, 4))) - assert(blockLocations2.contains(RDDBlockId(1, 0))) - assert(blockLocations2.contains(RDDBlockId(1, 1))) - assert(blockLocations2.contains(RDDBlockId(1, 2))) - assert(blockLocations2.size === 8) - assert(blockLocations2(RDDBlockId(0, 0)) === Seq("dog:1", "cat:3")) - assert(blockLocations2(RDDBlockId(0, 1)) === Seq("dog:1")) - assert(blockLocations2(RDDBlockId(0, 2)) === Seq("duck:2")) - assert(blockLocations2(RDDBlockId(0, 3)) === Seq("duck:2")) - assert(blockLocations2(RDDBlockId(1, 0)) === Seq("dog:1", "duck:2")) - assert(blockLocations2(RDDBlockId(1, 1)) === Seq("duck:2")) - assert(blockLocations2(RDDBlockId(0, 4)) === Seq("dog:1", "cat:3")) - assert(blockLocations2(RDDBlockId(1, 2)) === Seq("cat:3")) - } - - test("StorageUtils.filterByRDD") { - val storageStatuses = stockStorageStatuses - val filteredStorageStatuses0 = StorageUtils.filterByRDD(storageStatuses, 0) - val filteredStorageStatuses1 = StorageUtils.filterByRDD(storageStatuses, 1) - assert(filteredStorageStatuses0.size === 3) - assert(filteredStorageStatuses1.size === 2) + blockLocations0 = StorageUtils.getRDDBlockLocations(storageStatuses, 0) + blockLocations1 = StorageUtils.getRDDBlockLocations(storageStatuses, 1) + assert(blockLocations0.size === 5) + assert(blockLocations1.size === 3) + assert(blockLocations0.contains(RDDBlockId(0, 0))) + assert(blockLocations0.contains(RDDBlockId(0, 1))) + assert(blockLocations0.contains(RDDBlockId(0, 2))) + assert(blockLocations0.contains(RDDBlockId(0, 3))) + assert(blockLocations0.contains(RDDBlockId(0, 4))) + assert(blockLocations1.contains(RDDBlockId(1, 0))) + assert(blockLocations1.contains(RDDBlockId(1, 1))) + assert(blockLocations1.contains(RDDBlockId(1, 2))) + assert(blockLocations0(RDDBlockId(0, 0)) === Seq("dog:1", "cat:3")) + assert(blockLocations0(RDDBlockId(0, 1)) === Seq("dog:1")) + assert(blockLocations0(RDDBlockId(0, 2)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 3)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 4)) === Seq("dog:1", "cat:3")) + assert(blockLocations1(RDDBlockId(1, 0)) === Seq("dog:1", "duck:2")) + assert(blockLocations1(RDDBlockId(1, 1)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(1, 2)) === Seq("cat:3")) } } From 6fef86a68cbd284a970a731f05722abb9291122c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 31 Jul 2014 20:14:33 -0700 Subject: [PATCH 08/16] Add extensive tests for new code in StorageStatus This tests just about every single method in StorageStatus. In addition, a few methods in StorageStatus are changed to return Map instead of Seq. This commit also adds a few comments indicating that StorageStatus#blocks is expensive and there are cheaper alternatives. --- .../apache/spark/storage/StorageUtils.scala | 55 +++-- .../org/apache/spark/ui/storage/RDDPage.scala | 2 +- .../apache/spark/storage/StorageSuite.scala | 226 +++++++++++------- 3 files changed, 172 insertions(+), 111 deletions(-) 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 e1f2ae2f3457..d4e59d96be07 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -55,22 +55,24 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { */ def this(bmid: BlockManagerId, maxMem: Long, initialBlocks: Map[BlockId, BlockStatus]) { this(bmid, maxMem) - initialBlocks.foreach { case (blockId, blockStatus) => addBlock(blockId, blockStatus) } + initialBlocks.foreach { case (bid, bstatus) => addBlock(bid, bstatus) } } - /** Return the blocks stored in this block manager. */ - def blocks: Seq[(BlockId, BlockStatus)] = { - _nonRddBlocks.toSeq ++ rddBlocks.toSeq - } + /** + * Return the blocks stored in this block manager. + * + * Note that this is somewhat expensive, as it involves cloning the underlying maps and then + * concatenating them together. Much faster alternatives exist for common operations such as + * contains, get, and size. + */ + def blocks: Map[BlockId, BlockStatus] = _nonRddBlocks ++ rddBlocks /** Return the RDD blocks stored in this block manager. */ - def rddBlocks: Seq[(BlockId, BlockStatus)] = { - _rddBlocks.flatMap { case (_, blocks) => blocks }.toSeq - } + def rddBlocks: Map[BlockId, BlockStatus] = _rddBlocks.flatMap { case (_, blocks) => blocks } /** Return the blocks that belong to the given RDD stored in this block manager. */ - def rddBlocksById(rddId: Int): Seq[(BlockId, BlockStatus)] = { - _rddBlocks.get(rddId).map(_.toSeq).getOrElse(Seq.empty) + def rddBlocksById(rddId: Int): Map[BlockId, BlockStatus] = { + _rddBlocks.get(rddId).getOrElse(Map.empty) } /** Add the given block to this storage status. */ @@ -84,7 +86,9 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { } /** Update the given block in this storage status. If it doesn't already exist, add it. */ - def updateBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = addBlock(blockId, blockStatus) + def updateBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { + addBlock(blockId, blockStatus) + } /** Remove the given block from this storage status. */ def removeBlock(blockId: BlockId): Option[BlockStatus] = { @@ -107,7 +111,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** * Return whether the given block is stored in this block manager in O(1) time. - * Note that the alternative of doing this through `blocks` is O(blocks), which is much slower. + * Note that this is much faster than `this.blocks.contains`, which is O(blocks) time. */ def containsBlock(blockId: BlockId): Boolean = { blockId match { @@ -119,8 +123,21 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { } /** - * Return the number of blocks in O(R) time, where R is the number of distinct RDD IDs. - * Note that the alternative of doing this through `blocks` is O(blocks), which is much slower. + * Return the given block stored in this block manager in O(1) time. + * Note that this is much faster than `this.blocks.get`, which is O(blocks) time. + */ + def getBlock(blockId: BlockId): Option[BlockStatus] = { + blockId match { + case RDDBlockId(rddId, _) => + _rddBlocks.get(rddId).map(_.get(blockId)).flatten + case _ => + _nonRddBlocks.get(blockId) + } + } + + /** + * Return the number of blocks stored in this block manager in O(rdds) time. + * Note that this is much faster than `this.blocks.size`, which is O(blocks) time. */ def numBlocks: Int = { _nonRddBlocks.size + _rddBlocks.values.map(_.size).reduceOption(_ + _).getOrElse(0) @@ -142,10 +159,10 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { def diskUsedByRDD(rddId: Int): Long = diskUsed(rddBlocksById(rddId)) // Helper methods for computing memory and disk usages - private def memUsed(_blocks: Seq[(BlockId, BlockStatus)]): Long = - _blocks.map { case (_, s) => s.memSize }.reduceOption(_ + _).getOrElse(0L) - private def diskUsed(_blocks: Seq[(BlockId, BlockStatus)]): Long = - _blocks.map { case (_, s) => s.diskSize }.reduceOption(_ + _).getOrElse(0L) + private def memUsed(_blocks: Map[BlockId, BlockStatus]): Long = + _blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + private def diskUsed(_blocks: Map[BlockId, BlockStatus]): Long = + _blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) } /** Helper methods for storage-related objects. */ @@ -190,7 +207,7 @@ private[spark] object StorageUtils { /** * Return mapping from block ID to its locations for each block that belongs to the given RDD. */ - def getRDDBlockLocations( + def getRddBlockLocations( storageStatuses: Seq[StorageStatus], rddId: Int): Map[BlockId, Seq[String]] = { val blockLocations = new mutable.HashMap[BlockId, mutable.ListBuffer[String]] 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 2ae835e66363..eb0fd1ba3938 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 @@ -45,7 +45,7 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) // Block table - val blockLocations = StorageUtils.getRDDBlockLocations(storageStatusList, rddId) + val blockLocations = StorageUtils.getRddBlockLocations(storageStatusList, rddId) val blocks = storageStatusList .flatMap(_.rddBlocksById(rddId)) .sortWith(_._1.name < _._1.name) 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 50924e4ce9ee..e266494d97a7 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -25,152 +25,203 @@ import org.scalatest.FunSuite class StorageSuite extends FunSuite { private val memAndDisk = StorageLevel.MEMORY_AND_DISK - // A list of StorageStatuses with RDDs 0 and 1 cached on different block managers - private def stockStorageStatuses: Seq[StorageStatus] = { - val status1 = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) - val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2, 2), 2000L) - val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3, 3), 3000L) - status1.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status1.addBlock(RDDBlockId(0, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status2.addBlock(RDDBlockId(0, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status2.addBlock(RDDBlockId(0, 3), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status2.addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status2.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status3.addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status3.addBlock(RDDBlockId(1, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) - Seq(status1, status2, status3) - } - - // A list of RDDInfo for RDDs 0 and 1 - private def stockRDDInfos: Seq[RDDInfo] = { - val info0 = new RDDInfo(0, "0", 10, memAndDisk) - val info1 = new RDDInfo(1, "1", 3, memAndDisk) - Seq(info0, info1) - } - - test("storage status add/remove non-RDD blocks") { + // For testing add/update/removeBlock (for non-RDD blocks) + private def storageStatus1: StorageStatus = { val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) assert(status.blocks.isEmpty) assert(status.rddBlocks.isEmpty) assert(status.memUsed === 0) assert(status.memRemaining === 1000L) assert(status.diskUsed === 0) - - // Add a few blocks status.addBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 10L, 20L, 0L)) status.addBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 10L, 20L, 0L)) status.addBlock(TestBlockId("faa"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status + } + + test("storage status add non-RDD blocks") { + val status = storageStatus1 assert(status.blocks.size === 3) + assert(status.blocks.contains(TestBlockId("foo"))) + assert(status.blocks.contains(TestBlockId("fee"))) + assert(status.blocks.contains(TestBlockId("faa"))) assert(status.rddBlocks.isEmpty) assert(status.memUsed === 30L) assert(status.memRemaining === 970L) assert(status.diskUsed === 60L) + } - // Update a few existing blocks + test("storage status update non-RDD blocks") { + val status = storageStatus1 status.updateBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 50L, 100L, 0L)) status.updateBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 100L, 20L, 0L)) assert(status.blocks.size === 3) - assert(status.rddBlocks.isEmpty) assert(status.memUsed === 160L) assert(status.memRemaining === 840L) assert(status.diskUsed === 140L) + } - // Remove a few blocks + test("storage status remove non-RDD blocks") { + val status = storageStatus1 status.removeBlock(TestBlockId("foo")) status.removeBlock(TestBlockId("faa")) assert(status.blocks.size === 1) - assert(status.rddBlocks.isEmpty) - assert(status.memUsed === 100L) - assert(status.memRemaining === 900L) + assert(status.blocks.contains(TestBlockId("fee"))) + assert(status.memUsed === 10L) + assert(status.memRemaining === 990L) assert(status.diskUsed === 20L) } - test("storage status add/remove RDD blocks") { + // For testing add/update/remove/contains/getBlock and numBlocks + private def storageStatus2: StorageStatus = { val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) - assert(status.blocks.isEmpty) assert(status.rddBlocks.isEmpty) - - // Add a few blocks - status.addBlock(TestBlockId("DANGER"), BlockStatus(memAndDisk, 10L, 20L, 0L)) - status.addBlock(TestBlockId("MANGER"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(TestBlockId("man"), BlockStatus(memAndDisk, 10L, 20L, 0L)) status.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 10L, 20L, 0L)) status.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 100L, 200L, 0L)) status.addBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 10L, 20L, 0L)) status.addBlock(RDDBlockId(2, 3), BlockStatus(memAndDisk, 10L, 20L, 0L)) status.addBlock(RDDBlockId(2, 4), BlockStatus(memAndDisk, 10L, 40L, 0L)) + status + } + + test("storage status add RDD blocks") { + val status = storageStatus2 assert(status.blocks.size === 7) assert(status.rddBlocks.size === 5) + assert(status.rddBlocks.contains(RDDBlockId(0, 0))) + assert(status.rddBlocks.contains(RDDBlockId(1, 1))) + assert(status.rddBlocks.contains(RDDBlockId(2, 2))) + assert(status.rddBlocks.contains(RDDBlockId(2, 3))) + assert(status.rddBlocks.contains(RDDBlockId(2, 4))) assert(status.rddBlocksById(0).size === 1) - assert(status.rddBlocksById(0).head._1 === RDDBlockId(0, 0)) - assert(status.rddBlocksById(0).head._2 === BlockStatus(memAndDisk, 10L, 20L, 0L)) + assert(status.rddBlocksById(0).contains(RDDBlockId(0, 0))) assert(status.rddBlocksById(1).size === 1) - assert(status.rddBlocksById(1).head._1 === RDDBlockId(1, 1)) - assert(status.rddBlocksById(1).head._2 === BlockStatus(memAndDisk, 100L, 200L, 0L)) + assert(status.rddBlocksById(1).contains(RDDBlockId(1, 1))) assert(status.rddBlocksById(2).size === 3) - assert(status.rddBlocksById(2).head._1 === RDDBlockId(2, 2)) - assert(status.rddBlocksById(2).head._2 === BlockStatus(memAndDisk, 10L, 20L, 0L)) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 2))) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 3))) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 4))) assert(status.memUsedByRDD(0) === 10L) assert(status.memUsedByRDD(1) === 100L) assert(status.memUsedByRDD(2) === 30L) assert(status.diskUsedByRDD(0) === 20L) assert(status.diskUsedByRDD(1) === 200L) assert(status.diskUsedByRDD(2) === 80L) + } - // Update a few blocks - status.addBlock(TestBlockId("DANGER"), BlockStatus(memAndDisk, 5000L, 0L, 0L)) + test("storage status update RDD blocks") { + val status = storageStatus2 + status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 5000L, 0L, 0L)) status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)) status.updateBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 0L, 1000L, 0L)) assert(status.blocks.size === 7) assert(status.rddBlocks.size === 5) assert(status.rddBlocksById(0).size === 1) - assert(status.rddBlocksById(0).head._1 === RDDBlockId(0, 0)) - assert(status.rddBlocksById(0).head._2 === BlockStatus(memAndDisk, 0L, 0L, 0L)) + assert(status.rddBlocksById(1).size === 1) assert(status.rddBlocksById(2).size === 3) - assert(status.rddBlocksById(2).head._1 === RDDBlockId(2, 2)) - assert(status.rddBlocksById(2).head._2 === BlockStatus(memAndDisk, 0L, 1000L, 0L)) assert(status.memUsedByRDD(0) === 0L) assert(status.memUsedByRDD(1) === 100L) assert(status.memUsedByRDD(2) === 20L) assert(status.diskUsedByRDD(0) === 0L) assert(status.diskUsedByRDD(1) === 200L) assert(status.diskUsedByRDD(2) === 1060L) + } - // Remove a few blocks - status.removeBlock(TestBlockId("MANGER")) + test("storage status remove RDD blocks") { + val status = storageStatus2 + status.removeBlock(TestBlockId("man")) status.removeBlock(RDDBlockId(1, 1)) status.removeBlock(RDDBlockId(2, 2)) status.removeBlock(RDDBlockId(2, 4)) assert(status.blocks.size === 3) assert(status.rddBlocks.size === 2) + assert(status.rddBlocks.contains(RDDBlockId(0, 0))) + assert(status.rddBlocks.contains(RDDBlockId(2, 3))) assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(0).contains(RDDBlockId(0, 0))) assert(status.rddBlocksById(1).size === 0) assert(status.rddBlocksById(2).size === 1) - assert(status.memUsedByRDD(0) === 0L) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 3))) + assert(status.memUsedByRDD(0) === 10L) assert(status.memUsedByRDD(1) === 0L) assert(status.memUsedByRDD(2) === 10L) - assert(status.diskUsedByRDD(0) === 0L) + assert(status.diskUsedByRDD(0) === 20L) assert(status.diskUsedByRDD(1) === 0L) assert(status.diskUsedByRDD(2) === 20L) + } - // Add a few blocks again - status.addBlock(RDDBlockId(2, 10), BlockStatus(memAndDisk, 10L, 200000L, 0L)) - status.addBlock(RDDBlockId(3, 5), BlockStatus(memAndDisk, 10L, 200L, 0L)) - assert(status.blocks.size === 5) - assert(status.rddBlocks.size === 4) - assert(status.rddBlocksById(0).size === 1) - assert(status.rddBlocksById(1).size === 0) - assert(status.rddBlocksById(2).size === 2) - assert(status.rddBlocksById(3).size === 1) - assert(status.rddBlocksById(3).head._1 === RDDBlockId(3, 5)) - assert(status.rddBlocksById(3).head._2 === BlockStatus(memAndDisk, 10L, 200L, 0L)) - assert(status.memUsedByRDD(0) === 0L) - assert(status.memUsedByRDD(1) === 0L) - assert(status.memUsedByRDD(2) === 20L) - assert(status.memUsedByRDD(3) === 10L) - assert(status.diskUsedByRDD(0) === 0L) - assert(status.diskUsedByRDD(1) === 0L) - assert(status.diskUsedByRDD(2) === 200020L) - assert(status.diskUsedByRDD(3) === 200L) + test("storage status containsBlock") { + val status = storageStatus2 + // blocks that actually exist + assert(status.blocks.contains(TestBlockId("dan")) === status.containsBlock(TestBlockId("dan"))) + assert(status.blocks.contains(TestBlockId("man")) === status.containsBlock(TestBlockId("man"))) + assert(status.blocks.contains(RDDBlockId(0, 0)) === status.containsBlock(RDDBlockId(0, 0))) + assert(status.blocks.contains(RDDBlockId(1, 1)) === status.containsBlock(RDDBlockId(1, 1))) + assert(status.blocks.contains(RDDBlockId(2, 2)) === status.containsBlock(RDDBlockId(2, 2))) + assert(status.blocks.contains(RDDBlockId(2, 3)) === status.containsBlock(RDDBlockId(2, 3))) + assert(status.blocks.contains(RDDBlockId(2, 4)) === status.containsBlock(RDDBlockId(2, 4))) + // blocks that don't exist + assert(status.blocks.contains(TestBlockId("fan")) === status.containsBlock(TestBlockId("fan"))) + assert(status.blocks.contains(RDDBlockId(100, 0)) === status.containsBlock(RDDBlockId(100, 0))) + } + + test("storage status getBlock") { + val status = storageStatus2 + // blocks that actually exist + assert(status.blocks.get(TestBlockId("dan")) === status.getBlock(TestBlockId("dan"))) + assert(status.blocks.get(TestBlockId("man")) === status.getBlock(TestBlockId("man"))) + assert(status.blocks.get(RDDBlockId(0, 0)) === status.getBlock(RDDBlockId(0, 0))) + assert(status.blocks.get(RDDBlockId(1, 1)) === status.getBlock(RDDBlockId(1, 1))) + assert(status.blocks.get(RDDBlockId(2, 2)) === status.getBlock(RDDBlockId(2, 2))) + assert(status.blocks.get(RDDBlockId(2, 3)) === status.getBlock(RDDBlockId(2, 3))) + assert(status.blocks.get(RDDBlockId(2, 4)) === status.getBlock(RDDBlockId(2, 4))) + // blocks that don't exist + assert(status.blocks.get(TestBlockId("fan")) === status.getBlock(TestBlockId("fan"))) + assert(status.blocks.get(RDDBlockId(100, 0)) === status.getBlock(RDDBlockId(100, 0))) + } + + test("storage status numBlocks") { + val status = storageStatus2 + assert(status.blocks.size === status.numBlocks) + status.addBlock(RDDBlockId(4, 4), BlockStatus(memAndDisk, 0L, 0L, 100L)) + assert(status.blocks.size === status.numBlocks) + status.addBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L, 100L)) + assert(status.blocks.size === status.numBlocks) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 100L)) + assert(status.blocks.size === status.numBlocks) + // update a block that doesn't exist + status.updateBlock(RDDBlockId(100, 99), BlockStatus(memAndDisk, 0L, 0L, 100L)) + assert(status.blocks.size === status.numBlocks) + status.removeBlock(RDDBlockId(0, 0)) + assert(status.blocks.size === status.numBlocks) + // remove a block that doesn't exist + status.removeBlock(RDDBlockId(1000, 999)) + assert(status.blocks.size === status.numBlocks) + } + + // For testing StorageUtils.updateRddInfo and StorageUtils.getRddBlockLocations + private def stockStorageStatuses: Seq[StorageStatus] = { + val status1 = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) + val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2, 2), 2000L) + val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3, 3), 3000L) + status1.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status1.addBlock(RDDBlockId(0, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(0, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(0, 3), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status3.addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status3.addBlock(RDDBlockId(1, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) + Seq(status1, status2, status3) + } + + // For testing StorageUtils.updateRddInfo + private def stockRDDInfos: Seq[RDDInfo] = { + val info0 = new RDDInfo(0, "0", 10, memAndDisk) + val info1 = new RDDInfo(1, "1", 3, memAndDisk) + Seq(info0, info1) } test("StorageUtils.updateRddInfo") { @@ -205,16 +256,15 @@ class StorageSuite extends FunSuite { assert(rddInfos(1).memSize === 202L) assert(rddInfos(1).diskSize === 204L) - // Actually update storage statuses so we can chain the calls to rddInfoFromStorageStatus + // Actually update storage statuses so we can chain the calls to StorageUtils.updateRddInfo updatedBlocks1.foreach { case (bid, bstatus) => - val statusWithBlock = storageStatuses.find(_.containsBlock(bid)) - statusWithBlock match { + storageStatuses.find(_.containsBlock(bid)) match { case Some(s) => s.updateBlock(bid, bstatus) case None => storageStatuses(0).addBlock(bid, bstatus) // arbitrarily pick the first } } - // Drop all of RDD 1 + // Drop all of RDD 1, following previous updates val updatedBlocks2 = Seq( (RDDBlockId(1, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), (RDDBlockId(1, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), @@ -230,10 +280,10 @@ class StorageSuite extends FunSuite { assert(rddInfos(1).diskSize === 0L) } - test("StorageUtils.getBlockLocations") { + test("StorageUtils.getRddBlockLocations") { val storageStatuses = stockStorageStatuses - var blockLocations0 = StorageUtils.getRDDBlockLocations(storageStatuses, 0) - var blockLocations1 = StorageUtils.getRDDBlockLocations(storageStatuses, 1) + val blockLocations0 = StorageUtils.getRddBlockLocations(storageStatuses, 0) + val blockLocations1 = StorageUtils.getRddBlockLocations(storageStatuses, 1) assert(blockLocations0.size === 5) assert(blockLocations1.size === 3) assert(blockLocations0.contains(RDDBlockId(0, 0))) @@ -252,23 +302,17 @@ class StorageSuite extends FunSuite { assert(blockLocations1(RDDBlockId(1, 0)) === Seq("duck:2")) assert(blockLocations1(RDDBlockId(1, 1)) === Seq("duck:2")) assert(blockLocations1(RDDBlockId(1, 2)) === Seq("cat:3")) + } - // Multiple locations + test("StorageUtils.getRddBlockLocations with multiple locations") { + val storageStatuses = stockStorageStatuses storageStatuses(0).addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) storageStatuses(0).addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) storageStatuses(2).addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) - blockLocations0 = StorageUtils.getRDDBlockLocations(storageStatuses, 0) - blockLocations1 = StorageUtils.getRDDBlockLocations(storageStatuses, 1) + val blockLocations0 = StorageUtils.getRddBlockLocations(storageStatuses, 0) + val blockLocations1 = StorageUtils.getRddBlockLocations(storageStatuses, 1) assert(blockLocations0.size === 5) assert(blockLocations1.size === 3) - assert(blockLocations0.contains(RDDBlockId(0, 0))) - assert(blockLocations0.contains(RDDBlockId(0, 1))) - assert(blockLocations0.contains(RDDBlockId(0, 2))) - assert(blockLocations0.contains(RDDBlockId(0, 3))) - assert(blockLocations0.contains(RDDBlockId(0, 4))) - assert(blockLocations1.contains(RDDBlockId(1, 0))) - assert(blockLocations1.contains(RDDBlockId(1, 1))) - assert(blockLocations1.contains(RDDBlockId(1, 2))) assert(blockLocations0(RDDBlockId(0, 0)) === Seq("dog:1", "cat:3")) assert(blockLocations0(RDDBlockId(0, 1)) === Seq("dog:1")) assert(blockLocations0(RDDBlockId(0, 2)) === Seq("duck:2")) From 2c3ef6ab40f4b10490859b531f7f89240070f069 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 1 Aug 2014 00:04:12 -0700 Subject: [PATCH 09/16] Actually filter out only the relevant RDDs Prior to this commit, the changes in the PR actually demonstrate little performance improvement under all workloads. This is because we update all RDDInfos, rather than only the ones whose blocks are being updated. Thus, even though the new filter logic in StorageStatus is correct, we still iterate through all the RDD blocks every time a task has an updated block. This commit avoids this by only calling StorageLevel.updateRDDInfo on the RDDs that need to be updated. --- .../main/scala/org/apache/spark/ui/storage/StorageTab.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 854430d33724..797929a88d2f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -48,9 +48,11 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar /** Filter RDD info to include only those with cached partitions */ def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq - /** Update each RDD's info to reflect any updates in the RDD's storage status */ + /** Update the storage info of the RDDs whose blocks are among the given updated blocks */ private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { - StorageUtils.updateRddInfo(_rddInfoMap.values.toSeq, storageStatusList, updatedBlocks) + val rddIdsToUpdate = updatedBlocks.flatMap { case (bid, _) => bid.asRDDId.map(_.rddId) }.toSet + val rddInfosToUpdate = _rddInfoMap.values.toSeq.filter { s => rddIdsToUpdate.contains(s.id) } + StorageUtils.updateRddInfo(rddInfosToUpdate, storageStatusList, updatedBlocks) } /** From e080b9ee5f31e11ba44d16556a0cb80e28e2aef7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 1 Aug 2014 11:54:45 -0700 Subject: [PATCH 10/16] Reduce run time of StorageUtils.updateRddInfo to near constant With the existing changes in the PR, the StorageListener still needed to iterate through all the blocks within a single RDD. Although this is already much better than before, it is still slow if a single RDD has many partitions. This commit further reduces the run time of StorageUtils.updateRddInfo to near constant. It achieves this by incrementally updating the storage information of each RDD (memory, disk, tachyon sizes) incrementally, rather than all at once when the caller demands it. A preliminary benchmark shows that the event queue length never exceeds 600 even for caching 10000 partitions within a single RDD. An important TODO is to add tests for the new code, as well as for the StorageListener, the source of the storage information on the UI. --- .../storage/BlockManagerMasterActor.scala | 5 + .../apache/spark/storage/StorageUtils.scala | 153 ++++++++++++------ .../apache/spark/ui/storage/StorageTab.scala | 2 +- .../apache/spark/storage/StorageSuite.scala | 48 +----- 4 files changed, 114 insertions(+), 94 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 9b13f32fff78..c17cf55b0d90 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -424,6 +424,11 @@ case class BlockStatus( def isCached: Boolean = memSize + diskSize + tachyonSize > 0 } +@DeveloperApi +object BlockStatus { + def empty: BlockStatus = BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) +} + private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, 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 d4e59d96be07..7b73b0a7e35b 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -20,12 +20,15 @@ package org.apache.spark.storage import scala.collection.Map import scala.collection.mutable +import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: - * Storage information for each BlockManager. This class assumes BlockId and BlockStatus are - * immutable, such that the consumers of this class will not mutate the source of the information. + * Storage information for each BlockManager. + * + * This class assumes BlockId and BlockStatus are immutable, such that the consumers of this + * class cannot mutate the source of the information. Accesses are not thread-safe. */ @DeveloperApi class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { @@ -33,22 +36,29 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** * Internal representation of the blocks stored in this block manager. * - * Common consumption patterns of these blocks include - * (1) selecting all blocks, - * (2) selecting only RDD blocks or, - * (3) selecting only the blocks that belong to a specific RDD - * - * If we are only interested in a fraction of the blocks, as in (2) and (3), we should avoid - * linearly scanning through all the blocks, which could be expensive if there are thousands - * of blocks on each block manager. We achieve this by storing RDD blocks and non-RDD blocks - * separately. In particular, RDD blocks are stored in a map indexed by RDD IDs, so we can - * filter out the blocks of interest quickly. - * + * A common consumption pattern is to access only the blocks that belong to a specific RDD. + * For this use case, we should avoid linearly scanning through all the blocks, which could + * be expensive if there are thousands of blocks on each block manager. Thus, we need to store + * RDD blocks and non-RDD blocks separately. In particular, we store RDD blocks in a map + * indexed by RDD IDs, so we can filter out the blocks of interest quickly. + * These collections should only be mutated through the add/update/removeBlock methods. */ private val _rddBlocks = new mutable.HashMap[Int, mutable.Map[BlockId, BlockStatus]] private val _nonRddBlocks = new mutable.HashMap[BlockId, BlockStatus] + /** + * A map of storage information associated with each RDD. + * + * The key is the ID of the RDD, and the value is a 4-tuple of the following: + * (size in memory, size on disk, size in tachyon, storage level) + * + * This is updated incrementally on each block added, updated or removed, so as to avoid + * linearly scanning through all the blocks within an RDD if we're only interested in a + * given RDD's storage information. + */ + private val _rddStorageInfo = new mutable.HashMap[Int, (Long, Long, Long, StorageLevel)] + /** * Instantiate a StorageStatus with the given initial blocks. This essentially makes a copy of * the original blocks map such that the fate of this storage status is not tied to the source. @@ -79,6 +89,14 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { blockId match { case RDDBlockId(rddId, _) => + // Update the storage info of the RDD, keeping track of any existing status for this block + val oldBlockStatus = getBlock(blockId).getOrElse(BlockStatus.empty) + val changeInMem = blockStatus.memSize - oldBlockStatus.memSize + val changeInDisk = blockStatus.diskSize - oldBlockStatus.diskSize + val changeInTachyon = blockStatus.tachyonSize - oldBlockStatus.tachyonSize + val level = blockStatus.storageLevel + updateRddStorageInfo(rddId, changeInMem, changeInDisk, changeInTachyon, level) + // Actually add the block itself _rddBlocks.getOrElseUpdate(rddId, new mutable.HashMap)(blockId) = blockStatus case _ => _nonRddBlocks(blockId) = blockStatus @@ -94,6 +112,11 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { def removeBlock(blockId: BlockId): Option[BlockStatus] = { blockId match { case RDDBlockId(rddId, _) => + // Update the storage info of the RDD if the block to remove exists + getBlock(blockId).foreach { s => + updateRddStorageInfo(rddId, -s.memSize, -s.diskSize, -s.tachyonSize, StorageLevel.NONE) + } + // Actually remove the block, if it exists if (_rddBlocks.contains(rddId)) { val removed = _rddBlocks(rddId).remove(blockId) // If the given RDD has no more blocks left, remove the RDD @@ -136,33 +159,79 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { } /** - * Return the number of blocks stored in this block manager in O(rdds) time. + * Return the number of blocks stored in this block manager in O(RDDs) time. * Note that this is much faster than `this.blocks.size`, which is O(blocks) time. */ def numBlocks: Int = { _nonRddBlocks.size + _rddBlocks.values.map(_.size).reduceOption(_ + _).getOrElse(0) } + /** + * Return the number of RDD blocks stored in this block manager in O(RDDs) time. + * Note that this is much faster than `this.rddBlocks.size`, which is O(RDD blocks) time. + */ + def numRddBlocks: Int = _rddBlocks.keys.map(numRddBlocksById).reduceOption(_ + _).getOrElse(0) + + /** + * Return the number of blocks that belong to the given RDD in O(1) time. + * Note that this is much faster than `this.rddBlocksById(rddId).size`, which is + * O(blocks in this RDD) time. + */ + def numRddBlocksById(rddId: Int): Int = _rddBlocks.get(rddId).map(_.size).getOrElse(0) + /** Return the memory used by this block manager. */ - def memUsed: Long = memUsed(blocks) + def memUsed: Long = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) /** Return the memory used by the given RDD in this block manager. */ - def memUsedByRDD(rddId: Int): Long = memUsed(rddBlocksById(rddId)) + def memUsedByRDD(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._1).getOrElse(0L) /** Return the memory remaining in this block manager. */ def memRemaining: Long = maxMem - memUsed /** Return the disk space used by this block manager. */ - def diskUsed: Long = diskUsed(blocks) + def diskUsed: Long = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) /** Return the disk space used by the given RDD in this block manager. */ - def diskUsedByRDD(rddId: Int): Long = diskUsed(rddBlocksById(rddId)) + def diskUsedByRDD(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._2).getOrElse(0L) + + /** Return the off-heap space used by this block manager. */ + def offHeapUsed: Long = blocks.values.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) + + /** Return the off-heap space used by the given RDD in this block manager. */ + def offHeapUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._3).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(_._4) - // Helper methods for computing memory and disk usages - private def memUsed(_blocks: Map[BlockId, BlockStatus]): Long = - _blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - private def diskUsed(_blocks: Map[BlockId, BlockStatus]): Long = - _blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + /** + * Helper function to update the given RDD's storage information based on the + * (possibly negative) changes in memory, disk, and off-heap memory usages. + */ + private def updateRddStorageInfo( + rddId: Int, + changeInMem: Long, + changeInDisk: Long, + changeInTachyon: Long, + storageLevel: StorageLevel): Unit = { + val emptyRddInfo = (0L, 0L, 0L, StorageLevel.NONE) + val oldRddInfo = _rddStorageInfo.getOrElse(rddId, emptyRddInfo) + val newRddInfo = oldRddInfo match { + case (oldRddMem, oldRddDisk, oldRddTachyon, _) => + val newRddMem = math.max(oldRddMem + changeInMem, 0L) + val newRddDisk = math.max(oldRddDisk + changeInDisk, 0L) + val newRddTachyon = math.max(oldRddTachyon + changeInTachyon, 0L) + (newRddMem, newRddDisk, newRddTachyon, storageLevel) + case _ => + // Should never happen + throw new SparkException(s"Existing information for $rddId is not of expected type") + } + // If this RDD is no longer persisted, remove it + if (newRddInfo._1 + newRddInfo._2 + newRddInfo._3 == 0) { + _rddStorageInfo.remove(rddId) + } else { + _rddStorageInfo(rddId) = newRddInfo + } + } } /** Helper methods for storage-related objects. */ @@ -172,32 +241,20 @@ private[spark] object StorageUtils { * Update the given list of RDDInfo with the given list of storage statuses. * This method overwrites the old values stored in the RDDInfo's. */ - def updateRddInfo( - rddInfos: Seq[RDDInfo], - storageStatuses: Seq[StorageStatus], - updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Unit = { - + def updateRddInfo(rddInfos: Seq[RDDInfo], statuses: Seq[StorageStatus]): Unit = { rddInfos.foreach { rddInfo => val rddId = rddInfo.id - - // Collect all block statuses that belong to the given RDD - val newBlocks = updatedBlocks.filter { case (bid, _) => - bid.asRDDId.filter(_.rddId == rddId).isDefined - } - val newBlockIds = newBlocks.map { case (bid, _) => bid }.toSet - val oldBlocks = storageStatuses - .flatMap(_.rddBlocksById(rddId)) - .filter { case (bid, _) => !newBlockIds.contains(bid) } // avoid double counting - val blocks = (oldBlocks ++ newBlocks).map { case (_, bstatus) => bstatus } - val persistedBlocks = blocks.filter(_.isCached) - // Assume all blocks belonging to the same RDD have the same storage level - val storageLevel = blocks.headOption.map(_.storageLevel).getOrElse(StorageLevel.NONE) - val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) - val tachyonSize = persistedBlocks.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) + val storageLevel = statuses + .map(_.rddStorageLevel(rddId)).flatMap(s => s).headOption.getOrElse(StorageLevel.NONE) + val numCachedPartitions = statuses + .map(_.numRddBlocksById(rddId)).reduceOption(_ + _).getOrElse(0) + val memSize = statuses.map(_.memUsedByRDD(rddId)).reduceOption(_ + _).getOrElse(0L) + val diskSize = statuses.map(_.diskUsedByRDD(rddId)).reduceOption(_ + _).getOrElse(0L) + val tachyonSize = statuses.map(_.offHeapUsedByRdd(rddId)).reduceOption(_ + _).getOrElse(0L) + rddInfo.storageLevel = storageLevel - rddInfo.numCachedPartitions = persistedBlocks.length + rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize rddInfo.diskSize = diskSize rddInfo.tachyonSize = tachyonSize @@ -207,11 +264,9 @@ private[spark] object StorageUtils { /** * Return mapping from block ID to its locations for each block that belongs to the given RDD. */ - def getRddBlockLocations( - storageStatuses: Seq[StorageStatus], - rddId: Int): Map[BlockId, Seq[String]] = { + def getRddBlockLocations(statuses: Seq[StorageStatus], rddId: Int): Map[BlockId, Seq[String]] = { val blockLocations = new mutable.HashMap[BlockId, mutable.ListBuffer[String]] - storageStatuses.foreach { status => + statuses.foreach { status => status.rddBlocksById(rddId).foreach { case (bid, _) => val location = status.blockManagerId.hostPort blockLocations.getOrElseUpdate(bid, mutable.ListBuffer.empty) += location diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 797929a88d2f..6257e6f8e5c8 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -52,7 +52,7 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { val rddIdsToUpdate = updatedBlocks.flatMap { case (bid, _) => bid.asRDDId.map(_.rddId) }.toSet val rddInfosToUpdate = _rddInfoMap.values.toSeq.filter { s => rddIdsToUpdate.contains(s.id) } - StorageUtils.updateRddInfo(rddInfosToUpdate, storageStatusList, updatedBlocks) + StorageUtils.updateRddInfo(rddInfosToUpdate, storageStatusList) } /** 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 e266494d97a7..cfc0f5e49101 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -228,56 +228,16 @@ class StorageSuite extends FunSuite { val storageStatuses = stockStorageStatuses val rddInfos = stockRDDInfos StorageUtils.updateRddInfo(rddInfos, storageStatuses) + assert(rddInfos(0).storageLevel === memAndDisk) assert(rddInfos(0).numCachedPartitions === 5) assert(rddInfos(0).memSize === 5L) assert(rddInfos(0).diskSize === 10L) + assert(rddInfos(0).tachyonSize === 0L) + assert(rddInfos(1).storageLevel === memAndDisk) assert(rddInfos(1).numCachedPartitions === 3) assert(rddInfos(1).memSize === 3L) assert(rddInfos(1).diskSize === 6L) - } - - test("StorageUtils.updateRddInfo with updated blocks") { - val storageStatuses = stockStorageStatuses - val rddInfos = stockRDDInfos - - // Drop 3 blocks from RDD 0, and cache more of RDD 1 - val updatedBlocks1 = Seq( - (RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(0, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(0, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(1, 0), BlockStatus(memAndDisk, 100L, 100L, 0L)), - (RDDBlockId(1, 100), BlockStatus(memAndDisk, 100L, 100L, 0L)) - ) - StorageUtils.updateRddInfo(rddInfos, storageStatuses, updatedBlocks1) - assert(rddInfos(0).numCachedPartitions === 2) - assert(rddInfos(0).memSize === 2L) - assert(rddInfos(0).diskSize === 4L) - assert(rddInfos(1).numCachedPartitions === 4) - assert(rddInfos(1).memSize === 202L) - assert(rddInfos(1).diskSize === 204L) - - // Actually update storage statuses so we can chain the calls to StorageUtils.updateRddInfo - updatedBlocks1.foreach { case (bid, bstatus) => - storageStatuses.find(_.containsBlock(bid)) match { - case Some(s) => s.updateBlock(bid, bstatus) - case None => storageStatuses(0).addBlock(bid, bstatus) // arbitrarily pick the first - } - } - - // Drop all of RDD 1, following previous updates - val updatedBlocks2 = Seq( - (RDDBlockId(1, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(1, 1), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(1, 2), BlockStatus(memAndDisk, 0L, 0L, 0L)), - (RDDBlockId(1, 100), BlockStatus(memAndDisk, 0L, 0L, 0L)) - ) - StorageUtils.updateRddInfo(rddInfos, storageStatuses, updatedBlocks2) - assert(rddInfos(0).numCachedPartitions === 2) - assert(rddInfos(0).memSize === 2L) - assert(rddInfos(0).diskSize === 4L) - assert(rddInfos(1).numCachedPartitions === 0) - assert(rddInfos(1).memSize === 0L) - assert(rddInfos(1).diskSize === 0L) + assert(rddInfos(1).tachyonSize === 0L) } test("StorageUtils.getRddBlockLocations") { From 6970bc84f2cb601062a73638c27b470371f466e6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 1 Aug 2014 14:05:02 -0700 Subject: [PATCH 11/16] Add extensive tests for StorageListener and the new code in StorageUtils --- .../spark/storage/StorageStatusListener.scala | 3 +- .../apache/spark/storage/StorageUtils.scala | 36 ++-- .../apache/spark/ui/storage/StorageTab.scala | 2 +- .../apache/spark/storage/StorageSuite.scala | 125 +++++++++++-- .../spark/ui/storage/StorageTabSuite.scala | 165 ++++++++++++++++++ 5 files changed, 294 insertions(+), 37 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala 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 3966c33e3fb9..d9066f766476 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -35,8 +35,7 @@ class StorageStatusListener extends SparkListener { /** Update storage status list to reflect updated block statuses */ private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { - val filteredStatus = executorIdToStorageStatus.get(execId) - filteredStatus.foreach { storageStatus => + executorIdToStorageStatus.get(execId).foreach { storageStatus => updatedBlocks.foreach { case (blockId, updatedStatus) => if (updatedStatus.storageLevel == StorageLevel.NONE) { storageStatus.removeBlock(blockId) 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 7b73b0a7e35b..a886ffc7a3cf 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -77,7 +77,13 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { */ def blocks: Map[BlockId, BlockStatus] = _nonRddBlocks ++ rddBlocks - /** Return the RDD blocks stored in this block manager. */ + /** + * Return the RDD blocks stored in this block manager. + * + * Note that this is somewhat expensive, as it involves cloning the underlying maps and then + * concatenating them together. Much faster alternatives exist for common operations such as + * getting the memory, disk, and off-heap memory sizes occupied by this RDD. + */ def rddBlocks: Map[BlockId, BlockStatus] = _rddBlocks.flatMap { case (_, blocks) => blocks } /** Return the blocks that belong to the given RDD stored in this block manager. */ @@ -85,7 +91,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { _rddBlocks.get(rddId).getOrElse(Map.empty) } - /** Add the given block to this storage status. */ + /** Add the given block to this storage status. If it already exists, overwrite it. */ def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { blockId match { case RDDBlockId(rddId, _) => @@ -162,15 +168,13 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { * Return the number of blocks stored in this block manager in O(RDDs) time. * Note that this is much faster than `this.blocks.size`, which is O(blocks) time. */ - def numBlocks: Int = { - _nonRddBlocks.size + _rddBlocks.values.map(_.size).reduceOption(_ + _).getOrElse(0) - } + def numBlocks: Int = _nonRddBlocks.size + numRddBlocks /** * Return the number of RDD blocks stored in this block manager in O(RDDs) time. * Note that this is much faster than `this.rddBlocks.size`, which is O(RDD blocks) time. */ - def numRddBlocks: Int = _rddBlocks.keys.map(numRddBlocksById).reduceOption(_ + _).getOrElse(0) + def numRddBlocks: Int = _rddBlocks.values.map(_.size).reduceOption(_ + _).getOrElse(0) /** * Return the number of blocks that belong to the given RDD in O(1) time. @@ -182,32 +186,32 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** Return the memory used by this block manager. */ def memUsed: Long = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - /** Return the memory used by the given RDD in this block manager. */ - def memUsedByRDD(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._1).getOrElse(0L) - /** Return the memory remaining in this block manager. */ def memRemaining: Long = maxMem - memUsed /** Return the disk space used by this block manager. */ def diskUsed: Long = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) - /** Return the disk space used by the given RDD in this block manager. */ - def diskUsedByRDD(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._2).getOrElse(0L) - /** Return the off-heap space used by this block manager. */ def offHeapUsed: Long = blocks.values.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) - /** Return the off-heap space used by the given RDD in this block manager. */ + /** 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) + + /** 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) + + /** Return the off-heap space used by the given RDD in this block manager in O(1) time. */ def offHeapUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._3).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(_._4) /** - * Helper function to update the given RDD's storage information based on the - * (possibly negative) changes in memory, disk, and off-heap memory usages. + * Helper function to update the given RDD's storage information based on the (possibly + * negative) changes in memory, disk, and off-heap memory usages. This is exposed for testing. */ - private def updateRddStorageInfo( + private[spark] def updateRddStorageInfo( rddId: Int, changeInMem: Long, changeInDisk: Long, diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 6257e6f8e5c8..5f6740d49552 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -41,7 +41,7 @@ private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage" */ @DeveloperApi class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { - private val _rddInfoMap = mutable.Map[Int, RDDInfo]() + private[ui] val _rddInfoMap = mutable.Map[Int, RDDInfo]() // exposed for testing def storageStatusList = storageStatusListener.storageStatusList 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 cfc0f5e49101..4ac4ff5bccf7 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -24,18 +24,21 @@ import org.scalatest.FunSuite */ class StorageSuite extends FunSuite { private val memAndDisk = StorageLevel.MEMORY_AND_DISK + private val memOnly = StorageLevel.MEMORY_ONLY + private val diskOnly = StorageLevel.DISK_ONLY - // For testing add/update/removeBlock (for non-RDD blocks) + // For testing add, update, and remove (for non-RDD blocks) private def storageStatus1: StorageStatus = { val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) assert(status.blocks.isEmpty) assert(status.rddBlocks.isEmpty) - assert(status.memUsed === 0) + assert(status.memUsed === 0L) assert(status.memRemaining === 1000L) - assert(status.diskUsed === 0) - status.addBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 10L, 20L, 0L)) - status.addBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 10L, 20L, 0L)) - status.addBlock(TestBlockId("faa"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + assert(status.diskUsed === 0L) + assert(status.offHeapUsed === 0L) + status.addBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(TestBlockId("faa"), BlockStatus(memAndDisk, 10L, 20L, 1L)) status } @@ -49,16 +52,18 @@ class StorageSuite extends FunSuite { assert(status.memUsed === 30L) assert(status.memRemaining === 970L) assert(status.diskUsed === 60L) + assert(status.offHeapUsed === 3L) } test("storage status update non-RDD blocks") { val status = storageStatus1 - status.updateBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 50L, 100L, 0L)) + status.updateBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 50L, 100L, 1L)) status.updateBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 100L, 20L, 0L)) assert(status.blocks.size === 3) assert(status.memUsed === 160L) assert(status.memRemaining === 840L) assert(status.diskUsed === 140L) + assert(status.offHeapUsed === 2L) } test("storage status remove non-RDD blocks") { @@ -70,17 +75,18 @@ class StorageSuite extends FunSuite { assert(status.memUsed === 10L) assert(status.memRemaining === 990L) assert(status.diskUsed === 20L) + assert(status.offHeapUsed === 1L) } - // For testing add/update/remove/contains/getBlock and numBlocks + // 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, 1), 1000L) assert(status.rddBlocks.isEmpty) status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L, 0L)) status.addBlock(TestBlockId("man"), BlockStatus(memAndDisk, 10L, 20L, 0L)) - status.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 10L, 20L, 0L)) - status.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 100L, 200L, 0L)) - status.addBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 100L, 200L, 1L)) + status.addBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 10L, 20L, 1L)) status.addBlock(RDDBlockId(2, 3), BlockStatus(memAndDisk, 10L, 20L, 0L)) status.addBlock(RDDBlockId(2, 4), BlockStatus(memAndDisk, 10L, 40L, 0L)) status @@ -109,6 +115,19 @@ class StorageSuite extends FunSuite { assert(status.diskUsedByRDD(0) === 20L) assert(status.diskUsedByRDD(1) === 200L) assert(status.diskUsedByRDD(2) === 80L) + assert(status.offHeapUsedByRdd(0) === 1L) + assert(status.offHeapUsedByRdd(1) === 1L) + assert(status.offHeapUsedByRdd(2) === 1L) + assert(status.rddStorageLevel(0) === Some(memAndDisk)) + assert(status.rddStorageLevel(1) === Some(memAndDisk)) + assert(status.rddStorageLevel(2) === Some(memAndDisk)) + + // Verify default values for RDDs that don't exist + assert(status.rddBlocksById(10).isEmpty) + assert(status.memUsedByRDD(10) === 0L) + assert(status.diskUsedByRDD(10) === 0L) + assert(status.offHeapUsedByRdd(10) === 0L) + assert(status.rddStorageLevel(10) === None) } test("storage status update RDD blocks") { @@ -127,6 +146,9 @@ class StorageSuite extends FunSuite { assert(status.diskUsedByRDD(0) === 0L) assert(status.diskUsedByRDD(1) === 200L) assert(status.diskUsedByRDD(2) === 1060L) + assert(status.offHeapUsedByRdd(0) === 0L) + assert(status.offHeapUsedByRdd(1) === 1L) + assert(status.offHeapUsedByRdd(2) === 0L) } test("storage status remove RDD blocks") { @@ -150,6 +172,9 @@ class StorageSuite extends FunSuite { assert(status.diskUsedByRDD(0) === 20L) assert(status.diskUsedByRDD(1) === 0L) assert(status.diskUsedByRDD(2) === 20L) + assert(status.offHeapUsedByRdd(0) === 1L) + assert(status.offHeapUsedByRdd(1) === 0L) + assert(status.offHeapUsedByRdd(2) === 0L) } test("storage status containsBlock") { @@ -182,23 +207,87 @@ class StorageSuite extends FunSuite { assert(status.blocks.get(RDDBlockId(100, 0)) === status.getBlock(RDDBlockId(100, 0))) } - test("storage status numBlocks") { + test("storage status num[Rdd]Blocks") { val status = storageStatus2 assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + status.addBlock(TestBlockId("Foo"), BlockStatus(memAndDisk, 0L, 0L, 100L)) status.addBlock(RDDBlockId(4, 4), BlockStatus(memAndDisk, 0L, 0L, 100L)) - assert(status.blocks.size === status.numBlocks) status.addBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L, 100L)) assert(status.blocks.size === status.numBlocks) - status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 100L)) - assert(status.blocks.size === status.numBlocks) - // update a block that doesn't exist - status.updateBlock(RDDBlockId(100, 99), BlockStatus(memAndDisk, 0L, 0L, 100L)) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + status.updateBlock(TestBlockId("Foo"), BlockStatus(memAndDisk, 0L, 10L, 400L)) + status.updateBlock(RDDBlockId(4, 0), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.updateBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.updateBlock(RDDBlockId(10, 10), BlockStatus(memAndDisk, 0L, 0L, 100L)) assert(status.blocks.size === status.numBlocks) - status.removeBlock(RDDBlockId(0, 0)) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + assert(status.rddBlocksById(100).size === status.numRddBlocksById(100)) + status.removeBlock(RDDBlockId(4, 0)) + status.removeBlock(RDDBlockId(10, 10)) assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) // remove a block that doesn't exist status.removeBlock(RDDBlockId(1000, 999)) assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + assert(status.rddBlocksById(1000).size === status.numRddBlocksById(1000)) + } + + test("storage status updateRddStorageInfo") { + val status = storageStatus2 + // Positive delta + status.updateRddStorageInfo(0, 1000L, 1000L, 1000L, memOnly) + status.updateRddStorageInfo(1, 2000L, 2000L, 2000L, diskOnly) + status.updateRddStorageInfo(2, 3000L, 3000L, 3000L, memAndDisk) + assert(status.memUsedByRDD(0) === 1010L) + assert(status.memUsedByRDD(1) === 2100L) + assert(status.memUsedByRDD(2) === 3030L) + assert(status.diskUsedByRDD(0) === 1020L) + assert(status.diskUsedByRDD(1) === 2200L) + assert(status.diskUsedByRDD(2) === 3080L) + assert(status.offHeapUsedByRdd(0) === 1001L) + assert(status.offHeapUsedByRdd(1) === 2001L) + assert(status.offHeapUsedByRdd(2) === 3001L) + assert(status.rddStorageLevel(0) === Some(memOnly)) + assert(status.rddStorageLevel(1) === Some(diskOnly)) + assert(status.rddStorageLevel(2) === Some(memAndDisk)) + + // Negative delta + status.updateRddStorageInfo(0, -100L, -100L, -100L, memOnly) + status.updateRddStorageInfo(1, -200L, -200L, -200L, diskOnly) + status.updateRddStorageInfo(2, -300L, -300L, -300L, memAndDisk) + assert(status.memUsedByRDD(0) === 910L) + assert(status.memUsedByRDD(1) === 1900L) + assert(status.memUsedByRDD(2) === 2730L) + assert(status.diskUsedByRDD(0) === 920L) + assert(status.diskUsedByRDD(1) === 2000L) + assert(status.diskUsedByRDD(2) === 2780L) + assert(status.offHeapUsedByRdd(0) === 901L) + assert(status.offHeapUsedByRdd(1) === 1801L) + assert(status.offHeapUsedByRdd(2) === 2701L) + + // Negative delta so large that the RDDs are no longer persisted + status.updateRddStorageInfo(0, -10000L, -10000L, -10000L, memOnly) + status.updateRddStorageInfo(1, -20000L, -20000L, -20000L, diskOnly) + status.updateRddStorageInfo(2, -30000L, -30000L, -30000L, memAndDisk) + assert(status.memUsedByRDD(0) === 0L) + assert(status.memUsedByRDD(1) === 0L) + assert(status.memUsedByRDD(2) === 0L) + assert(status.diskUsedByRDD(0) === 0L) + assert(status.diskUsedByRDD(1) === 0L) + assert(status.diskUsedByRDD(2) === 0L) + assert(status.offHeapUsedByRdd(0) === 0L) + assert(status.offHeapUsedByRdd(1) === 0L) + assert(status.offHeapUsedByRdd(2) === 0L) } // For testing StorageUtils.updateRddInfo and StorageUtils.getRddBlockLocations diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala new file mode 100644 index 000000000000..a055a82bb2f3 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.storage + +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.Success +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler._ +import org.apache.spark.storage._ + +/** + * Test various functionality in the StorageListener that supports the StorageTab. + */ +class StorageTabSuite extends FunSuite with BeforeAndAfter { + private var bus: LiveListenerBus = _ + private var storageStatusListener: StorageStatusListener = _ + private var storageListener: StorageListener = _ + private val memAndDisk = StorageLevel.MEMORY_AND_DISK + private val memOnly = StorageLevel.MEMORY_ONLY + private val none = StorageLevel.NONE + private val taskInfo = new TaskInfo(0, 0, 0, 0, "big", "dog", TaskLocality.ANY, false) + private def rddInfo0 = new RDDInfo(0, "freedom", 100, memOnly) + private def rddInfo1 = new RDDInfo(1, "hostage", 200, memOnly) + private def rddInfo2 = new RDDInfo(2, "sanity", 300, memAndDisk) + private def rddInfo3 = new RDDInfo(3, "grace", 400, memAndDisk) + private val bm1 = BlockManagerId("big", "dog", 1, 1) + + before { + bus = new LiveListenerBus + storageStatusListener = new StorageStatusListener + storageListener = new StorageListener(storageStatusListener) + bus.addListener(storageStatusListener) + bus.addListener(storageListener) + } + + test("stage submitted / completed") { + assert(storageListener._rddInfoMap.isEmpty) + assert(storageListener.rddInfoList.isEmpty) + + // 2 RDDs are known, but none are cached + val stageInfo0 = new StageInfo(0, "0", 100, Seq(rddInfo0, rddInfo1), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 2) + assert(storageListener.rddInfoList.isEmpty) + + // 4 RDDs are known, but only 2 are cached + val rddInfo2Cached = rddInfo2 + val rddInfo3Cached = rddInfo3 + rddInfo2Cached.numCachedPartitions = 1 + rddInfo3Cached.numCachedPartitions = 1 + val stageInfo1 = new StageInfo(1, "0", 100, Seq(rddInfo2Cached, rddInfo3Cached), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) + assert(storageListener._rddInfoMap.size === 4) + assert(storageListener.rddInfoList.size === 2) + + // Submitting RDDInfos with duplicate IDs does nothing + val rddInfo0Cached = new RDDInfo(0, "freedom", 100, StorageLevel.MEMORY_ONLY) + rddInfo0Cached.numCachedPartitions = 1 + val stageInfo0Cached = new StageInfo(0, "0", 100, Seq(rddInfo0), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo0Cached)) + assert(storageListener._rddInfoMap.size === 4) + assert(storageListener.rddInfoList.size === 2) + + // We only keep around the RDDs that are cached + bus.postToAll(SparkListenerStageCompleted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 2) + assert(storageListener.rddInfoList.size === 2) + } + + test("unpersist") { + val rddInfo0Cached = rddInfo0 + val rddInfo1Cached = rddInfo1 + rddInfo0Cached.numCachedPartitions = 1 + rddInfo1Cached.numCachedPartitions = 1 + val stageInfo0 = new StageInfo(0, "0", 100, Seq(rddInfo0Cached, rddInfo1Cached), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 2) + assert(storageListener.rddInfoList.size === 2) + bus.postToAll(SparkListenerUnpersistRDD(0)) + assert(storageListener._rddInfoMap.size === 1) + assert(storageListener.rddInfoList.size === 1) + bus.postToAll(SparkListenerUnpersistRDD(4)) // doesn't exist + assert(storageListener._rddInfoMap.size === 1) + assert(storageListener.rddInfoList.size === 1) + bus.postToAll(SparkListenerUnpersistRDD(1)) + assert(storageListener._rddInfoMap.size === 0) + assert(storageListener.rddInfoList.size === 0) + } + + test("task end") { + val myRddInfo0 = rddInfo0 + val myRddInfo1 = rddInfo1 + val myRddInfo2 = rddInfo2 + val stageInfo0 = new StageInfo(0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") + bus.postToAll(SparkListenerBlockManagerAdded(bm1, 1000L)) + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 3) + assert(storageListener.rddInfoList.size === 0) // not cached + assert(!storageListener._rddInfoMap(0).isCached) + assert(!storageListener._rddInfoMap(1).isCached) + assert(!storageListener._rddInfoMap(2).isCached) + + // Task end with no updated blocks. This should not change anything. + bus.postToAll(SparkListenerTaskEnd(0, "obliteration", Success, taskInfo, new TaskMetrics)) + assert(storageListener._rddInfoMap.size === 3) + assert(storageListener.rddInfoList.size === 0) + + // Task end with a few new persisted blocks, some from the same RDD + val metrics1 = new TaskMetrics + metrics1.updatedBlocks = Some(Seq( + (RDDBlockId(0, 100), BlockStatus(memAndDisk, 400L, 0L, 0L)), + (RDDBlockId(0, 101), BlockStatus(memAndDisk, 0L, 400L, 0L)), + (RDDBlockId(0, 102), BlockStatus(memAndDisk, 400L, 0L, 200L)), + (RDDBlockId(1, 20), BlockStatus(memAndDisk, 0L, 240L, 0L)) + )) + bus.postToAll(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo, metrics1)) + assert(storageListener._rddInfoMap(0).memSize === 800L) + assert(storageListener._rddInfoMap(0).diskSize === 400L) + assert(storageListener._rddInfoMap(0).tachyonSize === 200L) + assert(storageListener._rddInfoMap(0).numCachedPartitions === 3) + assert(storageListener._rddInfoMap(0).isCached) + assert(storageListener._rddInfoMap(1).memSize === 0L) + assert(storageListener._rddInfoMap(1).diskSize === 240L) + assert(storageListener._rddInfoMap(1).tachyonSize === 0L) + assert(storageListener._rddInfoMap(1).numCachedPartitions === 1) + assert(storageListener._rddInfoMap(1).isCached) + assert(!storageListener._rddInfoMap(2).isCached) + assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) + + // Task end with a few dropped blocks + val metrics2 = new TaskMetrics + metrics2.updatedBlocks = Some(Seq( + (RDDBlockId(0, 100), BlockStatus(none, 0L, 0L, 0L)), + (RDDBlockId(1, 20), BlockStatus(none, 0L, 0L, 0L)), + (RDDBlockId(2, 40), BlockStatus(none, 0L, 0L, 0L)), // doesn't actually exist + (RDDBlockId(4, 80), BlockStatus(none, 0L, 0L, 0L)) // doesn't actually exist + )) + bus.postToAll(SparkListenerTaskEnd(2, "obliteration", Success, taskInfo, metrics2)) + assert(storageListener._rddInfoMap(0).memSize === 400L) + assert(storageListener._rddInfoMap(0).diskSize === 400L) + assert(storageListener._rddInfoMap(0).tachyonSize === 200L) + assert(storageListener._rddInfoMap(0).numCachedPartitions === 2) + assert(storageListener._rddInfoMap(0).isCached) + assert(!storageListener._rddInfoMap(1).isCached) + assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) + assert(!storageListener._rddInfoMap(2).isCached) + assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) + } + +} \ No newline at end of file From af19bc05bd148df04575ec1c905e658f5dc6d2bf Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 1 Aug 2014 14:06:51 -0700 Subject: [PATCH 12/16] *UsedByRDD -> *UsedByRdd (minor) --- .../apache/spark/storage/StorageUtils.scala | 8 +- .../org/apache/spark/ui/storage/RDDPage.scala | 4 +- .../apache/spark/storage/StorageSuite.scala | 76 +++++++++---------- 3 files changed, 44 insertions(+), 44 deletions(-) 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 a886ffc7a3cf..09871c2b87c1 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -196,10 +196,10 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { def offHeapUsed: Long = blocks.values.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) /** 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(_._1).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(_._2).getOrElse(0L) /** Return the off-heap space used by the given RDD in this block manager in O(1) time. */ def offHeapUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._3).getOrElse(0L) @@ -253,8 +253,8 @@ private[spark] object StorageUtils { .map(_.rddStorageLevel(rddId)).flatMap(s => s).headOption.getOrElse(StorageLevel.NONE) val numCachedPartitions = statuses .map(_.numRddBlocksById(rddId)).reduceOption(_ + _).getOrElse(0) - val memSize = statuses.map(_.memUsedByRDD(rddId)).reduceOption(_ + _).getOrElse(0L) - val diskSize = statuses.map(_.diskUsedByRDD(rddId)).reduceOption(_ + _).getOrElse(0L) + val memSize = statuses.map(_.memUsedByRdd(rddId)).reduceOption(_ + _).getOrElse(0L) + val diskSize = statuses.map(_.diskUsedByRdd(rddId)).reduceOption(_ + _).getOrElse(0L) val tachyonSize = statuses.map(_.offHeapUsedByRdd(rddId)).reduceOption(_ + _).getOrElse(0L) rddInfo.storageLevel = storageLevel 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 eb0fd1ba3938..e2fea872cd4d 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 @@ -120,10 +120,10 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { {status.blockManagerId.host + ":" + status.blockManagerId.port} - {Utils.bytesToString(status.memUsedByRDD(rddId))} + {Utils.bytesToString(status.memUsedByRdd(rddId))} ({Utils.bytesToString(status.memRemaining)} Remaining) - {Utils.bytesToString(status.diskUsedByRDD(rddId))} + {Utils.bytesToString(status.diskUsedByRdd(rddId))} } 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 4ac4ff5bccf7..50f885e05cfc 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -109,12 +109,12 @@ class StorageSuite extends FunSuite { assert(status.rddBlocksById(2).contains(RDDBlockId(2, 2))) assert(status.rddBlocksById(2).contains(RDDBlockId(2, 3))) assert(status.rddBlocksById(2).contains(RDDBlockId(2, 4))) - assert(status.memUsedByRDD(0) === 10L) - assert(status.memUsedByRDD(1) === 100L) - assert(status.memUsedByRDD(2) === 30L) - assert(status.diskUsedByRDD(0) === 20L) - assert(status.diskUsedByRDD(1) === 200L) - assert(status.diskUsedByRDD(2) === 80L) + assert(status.memUsedByRdd(0) === 10L) + assert(status.memUsedByRdd(1) === 100L) + assert(status.memUsedByRdd(2) === 30L) + assert(status.diskUsedByRdd(0) === 20L) + assert(status.diskUsedByRdd(1) === 200L) + assert(status.diskUsedByRdd(2) === 80L) assert(status.offHeapUsedByRdd(0) === 1L) assert(status.offHeapUsedByRdd(1) === 1L) assert(status.offHeapUsedByRdd(2) === 1L) @@ -124,8 +124,8 @@ class StorageSuite extends FunSuite { // Verify default values for RDDs that don't exist assert(status.rddBlocksById(10).isEmpty) - assert(status.memUsedByRDD(10) === 0L) - assert(status.diskUsedByRDD(10) === 0L) + assert(status.memUsedByRdd(10) === 0L) + assert(status.diskUsedByRdd(10) === 0L) assert(status.offHeapUsedByRdd(10) === 0L) assert(status.rddStorageLevel(10) === None) } @@ -140,12 +140,12 @@ class StorageSuite extends FunSuite { assert(status.rddBlocksById(0).size === 1) assert(status.rddBlocksById(1).size === 1) assert(status.rddBlocksById(2).size === 3) - assert(status.memUsedByRDD(0) === 0L) - assert(status.memUsedByRDD(1) === 100L) - assert(status.memUsedByRDD(2) === 20L) - assert(status.diskUsedByRDD(0) === 0L) - assert(status.diskUsedByRDD(1) === 200L) - assert(status.diskUsedByRDD(2) === 1060L) + assert(status.memUsedByRdd(0) === 0L) + assert(status.memUsedByRdd(1) === 100L) + assert(status.memUsedByRdd(2) === 20L) + assert(status.diskUsedByRdd(0) === 0L) + assert(status.diskUsedByRdd(1) === 200L) + assert(status.diskUsedByRdd(2) === 1060L) assert(status.offHeapUsedByRdd(0) === 0L) assert(status.offHeapUsedByRdd(1) === 1L) assert(status.offHeapUsedByRdd(2) === 0L) @@ -166,12 +166,12 @@ class StorageSuite extends FunSuite { assert(status.rddBlocksById(1).size === 0) assert(status.rddBlocksById(2).size === 1) assert(status.rddBlocksById(2).contains(RDDBlockId(2, 3))) - assert(status.memUsedByRDD(0) === 10L) - assert(status.memUsedByRDD(1) === 0L) - assert(status.memUsedByRDD(2) === 10L) - assert(status.diskUsedByRDD(0) === 20L) - assert(status.diskUsedByRDD(1) === 0L) - assert(status.diskUsedByRDD(2) === 20L) + assert(status.memUsedByRdd(0) === 10L) + assert(status.memUsedByRdd(1) === 0L) + assert(status.memUsedByRdd(2) === 10L) + assert(status.diskUsedByRdd(0) === 20L) + assert(status.diskUsedByRdd(1) === 0L) + assert(status.diskUsedByRdd(2) === 20L) assert(status.offHeapUsedByRdd(0) === 1L) assert(status.offHeapUsedByRdd(1) === 0L) assert(status.offHeapUsedByRdd(2) === 0L) @@ -248,12 +248,12 @@ class StorageSuite extends FunSuite { status.updateRddStorageInfo(0, 1000L, 1000L, 1000L, memOnly) status.updateRddStorageInfo(1, 2000L, 2000L, 2000L, diskOnly) status.updateRddStorageInfo(2, 3000L, 3000L, 3000L, memAndDisk) - assert(status.memUsedByRDD(0) === 1010L) - assert(status.memUsedByRDD(1) === 2100L) - assert(status.memUsedByRDD(2) === 3030L) - assert(status.diskUsedByRDD(0) === 1020L) - assert(status.diskUsedByRDD(1) === 2200L) - assert(status.diskUsedByRDD(2) === 3080L) + assert(status.memUsedByRdd(0) === 1010L) + assert(status.memUsedByRdd(1) === 2100L) + assert(status.memUsedByRdd(2) === 3030L) + assert(status.diskUsedByRdd(0) === 1020L) + assert(status.diskUsedByRdd(1) === 2200L) + assert(status.diskUsedByRdd(2) === 3080L) assert(status.offHeapUsedByRdd(0) === 1001L) assert(status.offHeapUsedByRdd(1) === 2001L) assert(status.offHeapUsedByRdd(2) === 3001L) @@ -265,12 +265,12 @@ class StorageSuite extends FunSuite { status.updateRddStorageInfo(0, -100L, -100L, -100L, memOnly) status.updateRddStorageInfo(1, -200L, -200L, -200L, diskOnly) status.updateRddStorageInfo(2, -300L, -300L, -300L, memAndDisk) - assert(status.memUsedByRDD(0) === 910L) - assert(status.memUsedByRDD(1) === 1900L) - assert(status.memUsedByRDD(2) === 2730L) - assert(status.diskUsedByRDD(0) === 920L) - assert(status.diskUsedByRDD(1) === 2000L) - assert(status.diskUsedByRDD(2) === 2780L) + assert(status.memUsedByRdd(0) === 910L) + assert(status.memUsedByRdd(1) === 1900L) + assert(status.memUsedByRdd(2) === 2730L) + assert(status.diskUsedByRdd(0) === 920L) + assert(status.diskUsedByRdd(1) === 2000L) + assert(status.diskUsedByRdd(2) === 2780L) assert(status.offHeapUsedByRdd(0) === 901L) assert(status.offHeapUsedByRdd(1) === 1801L) assert(status.offHeapUsedByRdd(2) === 2701L) @@ -279,12 +279,12 @@ class StorageSuite extends FunSuite { status.updateRddStorageInfo(0, -10000L, -10000L, -10000L, memOnly) status.updateRddStorageInfo(1, -20000L, -20000L, -20000L, diskOnly) status.updateRddStorageInfo(2, -30000L, -30000L, -30000L, memAndDisk) - assert(status.memUsedByRDD(0) === 0L) - assert(status.memUsedByRDD(1) === 0L) - assert(status.memUsedByRDD(2) === 0L) - assert(status.diskUsedByRDD(0) === 0L) - assert(status.diskUsedByRDD(1) === 0L) - assert(status.diskUsedByRDD(2) === 0L) + assert(status.memUsedByRdd(0) === 0L) + assert(status.memUsedByRdd(1) === 0L) + assert(status.memUsedByRdd(2) === 0L) + assert(status.diskUsedByRdd(0) === 0L) + assert(status.diskUsedByRdd(1) === 0L) + assert(status.diskUsedByRdd(2) === 0L) assert(status.offHeapUsedByRdd(0) === 0L) assert(status.offHeapUsedByRdd(1) === 0L) assert(status.offHeapUsedByRdd(2) === 0L) From bf6f09bef02cff842e806776a2552f7a91f7f440 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 1 Aug 2014 17:16:00 -0700 Subject: [PATCH 13/16] Minor changes --- .../main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala | 6 +++--- .../scala/org/apache/spark/ui/storage/StorageTabSuite.scala | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) 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 cc8ef9f1b612..8b18627a17e1 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 @@ -51,7 +51,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) val memUsed = storageStatusList.map(_.memUsed).fold(0L)(_ + _) - val diskSpaceUsed = storageStatusList.map(_.diskUsed).reduceOption(_ + _).getOrElse(0L) + val diskUsed = storageStatusList.map(_.diskUsed).fold(0L)(_ + _) val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.id) @@ -80,7 +80,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { - {execInfoSorted.map(execRow(_))} + {execInfoSorted.map(execRow)} @@ -91,7 +91,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") {
  • Memory: {Utils.bytesToString(memUsed)} Used ({Utils.bytesToString(maxMem)} Total)
  • -
  • Disk: {Utils.bytesToString(diskSpaceUsed)} Used
  • +
  • Disk: {Utils.bytesToString(diskUsed)} Used
  • diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index a055a82bb2f3..6e68dcb3425a 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -162,4 +162,4 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) } -} \ No newline at end of file +} From a91be462abe012371904585c8c585386012f86a1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 1 Aug 2014 18:50:20 -0700 Subject: [PATCH 14/16] Make ExecutorsPage blazingly fast This commit incrementally keeps track of storage information for non-RDD blocks in addition to RDD blocks, which we currently already do. This allows us to find the memory, disk, and off-heap memory used by an executor without iterating through all of its blocks. --- .../apache/spark/storage/StorageUtils.scala | 119 ++++++++++-------- .../apache/spark/storage/StorageSuite.scala | 73 ++++------- 2 files changed, 91 insertions(+), 101 deletions(-) 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 09871c2b87c1..07cfcfec1c86 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -20,7 +20,6 @@ package org.apache.spark.storage import scala.collection.Map import scala.collection.mutable -import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi /** @@ -48,16 +47,20 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { private val _nonRddBlocks = new mutable.HashMap[BlockId, BlockStatus] /** - * A map of storage information associated with each RDD. + * Storage information of the blocks that entails memory, disk, and off-heap memory usage. * - * The key is the ID of the RDD, and the value is a 4-tuple of the following: - * (size in memory, size on disk, size in tachyon, storage level) + * 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: * - * This is updated incrementally on each block added, updated or removed, so as to avoid - * linearly scanning through all the blocks within an RDD if we're only interested in a - * given RDD's storage information. + * (memory size, disk size, off-heap 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, Long, StorageLevel)] + private var _nonRddStorageInfo: (Long, Long, Long) = (0L, 0L, 0L) /** * Instantiate a StorageStatus with the given initial blocks. This essentially makes a copy of @@ -93,16 +96,9 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** Add the given block to this storage status. If it already exists, overwrite it. */ def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { + updateStorageInfo(blockId, blockStatus) blockId match { case RDDBlockId(rddId, _) => - // Update the storage info of the RDD, keeping track of any existing status for this block - val oldBlockStatus = getBlock(blockId).getOrElse(BlockStatus.empty) - val changeInMem = blockStatus.memSize - oldBlockStatus.memSize - val changeInDisk = blockStatus.diskSize - oldBlockStatus.diskSize - val changeInTachyon = blockStatus.tachyonSize - oldBlockStatus.tachyonSize - val level = blockStatus.storageLevel - updateRddStorageInfo(rddId, changeInMem, changeInDisk, changeInTachyon, level) - // Actually add the block itself _rddBlocks.getOrElseUpdate(rddId, new mutable.HashMap)(blockId) = blockStatus case _ => _nonRddBlocks(blockId) = blockStatus @@ -116,12 +112,9 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** Remove the given block from this storage status. */ def removeBlock(blockId: BlockId): Option[BlockStatus] = { + updateStorageInfo(blockId, BlockStatus.empty) blockId match { case RDDBlockId(rddId, _) => - // Update the storage info of the RDD if the block to remove exists - getBlock(blockId).foreach { s => - updateRddStorageInfo(rddId, -s.memSize, -s.diskSize, -s.tachyonSize, StorageLevel.NONE) - } // Actually remove the block, if it exists if (_rddBlocks.contains(rddId)) { val removed = _rddBlocks(rddId).remove(blockId) @@ -145,7 +138,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { def containsBlock(blockId: BlockId): Boolean = { blockId match { case RDDBlockId(rddId, _) => - _rddBlocks.get(rddId).filter(_.contains(blockId)).isDefined + _rddBlocks.get(rddId).exists(_.contains(blockId)) case _ => _nonRddBlocks.contains(blockId) } @@ -174,7 +167,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { * Return the number of RDD blocks stored in this block manager in O(RDDs) time. * Note that this is much faster than `this.rddBlocks.size`, which is O(RDD blocks) time. */ - def numRddBlocks: Int = _rddBlocks.values.map(_.size).reduceOption(_ + _).getOrElse(0) + def numRddBlocks: Int = _rddBlocks.values.map(_.size).fold(0)(_ + _) /** * Return the number of blocks that belong to the given RDD in O(1) time. @@ -183,17 +176,20 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { */ def numRddBlocksById(rddId: Int): Int = _rddBlocks.get(rddId).map(_.size).getOrElse(0) - /** Return the memory used by this block manager. */ - def memUsed: Long = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - /** Return the memory remaining in this block manager. */ def memRemaining: Long = maxMem - memUsed + /** Return the memory used by this block manager. */ + def memUsed: Long = + _nonRddStorageInfo._1 + _rddBlocks.keys.toSeq.map(memUsedByRdd).fold(0L)(_ + _) + /** Return the disk space used by this block manager. */ - def diskUsed: Long = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + def diskUsed: Long = + _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).fold(0L)(_ + _) /** Return the off-heap space used by this block manager. */ - def offHeapUsed: Long = blocks.values.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) + def offHeapUsed: Long = + _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(offHeapUsedByRdd).fold(0L)(_ + _) /** 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) @@ -208,34 +204,50 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { def rddStorageLevel(rddId: Int): Option[StorageLevel] = _rddStorageInfo.get(rddId).map(_._4) /** - * Helper function to update the given RDD's storage information based on the (possibly - * negative) changes in memory, disk, and off-heap memory usages. This is exposed for testing. + * Update the relevant storage info, taking into account any existing status for this block. + * This is exposed for testing. */ - private[spark] def updateRddStorageInfo( - rddId: Int, - changeInMem: Long, - changeInDisk: Long, - changeInTachyon: Long, - storageLevel: StorageLevel): Unit = { - val emptyRddInfo = (0L, 0L, 0L, StorageLevel.NONE) - val oldRddInfo = _rddStorageInfo.getOrElse(rddId, emptyRddInfo) - val newRddInfo = oldRddInfo match { - case (oldRddMem, oldRddDisk, oldRddTachyon, _) => - val newRddMem = math.max(oldRddMem + changeInMem, 0L) - val newRddDisk = math.max(oldRddDisk + changeInDisk, 0L) - val newRddTachyon = math.max(oldRddTachyon + changeInTachyon, 0L) - (newRddMem, newRddDisk, newRddTachyon, storageLevel) + private[spark] def updateStorageInfo(blockId: BlockId, newBlockStatus: BlockStatus): Unit = { + val oldBlockStatus = getBlock(blockId).getOrElse(BlockStatus.empty) + val changeInMem = newBlockStatus.memSize - oldBlockStatus.memSize + val changeInDisk = newBlockStatus.diskSize - oldBlockStatus.diskSize + val changeInTachyon = newBlockStatus.tachyonSize - oldBlockStatus.tachyonSize + val level = newBlockStatus.storageLevel + + // Compute new info from old info + val oldInfo: (Long, Long, Long) = blockId match { + case RDDBlockId(rddId, _) => + _rddStorageInfo.get(rddId) + .map { case (mem, disk, tachyon, _) => (mem, disk, tachyon) } + .getOrElse((0L, 0L, 0L)) case _ => - // Should never happen - throw new SparkException(s"Existing information for $rddId is not of expected type") + _nonRddStorageInfo } - // If this RDD is no longer persisted, remove it - if (newRddInfo._1 + newRddInfo._2 + newRddInfo._3 == 0) { - _rddStorageInfo.remove(rddId) - } else { - _rddStorageInfo(rddId) = newRddInfo + val newInfo: (Long, Long, Long) = oldInfo match { + case (oldMem, oldDisk, oldTachyon) => + val newMem = math.max(oldMem + changeInMem, 0L) + val newDisk = math.max(oldDisk + changeInDisk, 0L) + val newTachyon = math.max(oldTachyon + changeInTachyon, 0L) + (newMem, newDisk, newTachyon) + } + + // Set the correct info + blockId match { + case RDDBlockId(rddId, _) => + newInfo match { + case (mem, disk, tachyon) => + // If this RDD is no longer persisted, remove it + if (mem + disk + tachyon == 0) { + _rddStorageInfo.remove(rddId) + } else { + _rddStorageInfo(rddId) = (mem, disk, tachyon, level) + } + } + case _ => + _nonRddStorageInfo = newInfo } } + } /** Helper methods for storage-related objects. */ @@ -251,11 +263,10 @@ private[spark] object StorageUtils { // Assume all blocks belonging to the same RDD have the same storage level val storageLevel = statuses .map(_.rddStorageLevel(rddId)).flatMap(s => s).headOption.getOrElse(StorageLevel.NONE) - val numCachedPartitions = statuses - .map(_.numRddBlocksById(rddId)).reduceOption(_ + _).getOrElse(0) - val memSize = statuses.map(_.memUsedByRdd(rddId)).reduceOption(_ + _).getOrElse(0L) - val diskSize = statuses.map(_.diskUsedByRdd(rddId)).reduceOption(_ + _).getOrElse(0L) - val tachyonSize = statuses.map(_.offHeapUsedByRdd(rddId)).reduceOption(_ + _).getOrElse(0L) + val numCachedPartitions = statuses.map(_.numRddBlocksById(rddId)).fold(0)(_ + _) + val memSize = statuses.map(_.memUsedByRdd(rddId)).fold(0L)(_ + _) + val diskSize = statuses.map(_.diskUsedByRdd(rddId)).fold(0L)(_ + _) + val tachyonSize = statuses.map(_.offHeapUsedByRdd(rddId)).fold(0L)(_ + _) rddInfo.storageLevel = storageLevel rddInfo.numCachedPartitions = numCachedPartitions 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 50f885e05cfc..4509d7a47f50 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -24,8 +24,6 @@ import org.scalatest.FunSuite */ class StorageSuite extends FunSuite { private val memAndDisk = StorageLevel.MEMORY_AND_DISK - private val memOnly = StorageLevel.MEMORY_ONLY - private val diskOnly = StorageLevel.DISK_ONLY // For testing add, update, and remove (for non-RDD blocks) private def storageStatus1: StorageStatus = { @@ -242,52 +240,33 @@ class StorageSuite extends FunSuite { assert(status.rddBlocksById(1000).size === status.numRddBlocksById(1000)) } - test("storage status updateRddStorageInfo") { + test("storage status memUsed, diskUsed, tachyonUsed") { val status = storageStatus2 - // Positive delta - status.updateRddStorageInfo(0, 1000L, 1000L, 1000L, memOnly) - status.updateRddStorageInfo(1, 2000L, 2000L, 2000L, diskOnly) - status.updateRddStorageInfo(2, 3000L, 3000L, 3000L, memAndDisk) - assert(status.memUsedByRdd(0) === 1010L) - assert(status.memUsedByRdd(1) === 2100L) - assert(status.memUsedByRdd(2) === 3030L) - assert(status.diskUsedByRdd(0) === 1020L) - assert(status.diskUsedByRdd(1) === 2200L) - assert(status.diskUsedByRdd(2) === 3080L) - assert(status.offHeapUsedByRdd(0) === 1001L) - assert(status.offHeapUsedByRdd(1) === 2001L) - assert(status.offHeapUsedByRdd(2) === 3001L) - assert(status.rddStorageLevel(0) === Some(memOnly)) - assert(status.rddStorageLevel(1) === Some(diskOnly)) - assert(status.rddStorageLevel(2) === Some(memAndDisk)) - - // Negative delta - status.updateRddStorageInfo(0, -100L, -100L, -100L, memOnly) - status.updateRddStorageInfo(1, -200L, -200L, -200L, diskOnly) - status.updateRddStorageInfo(2, -300L, -300L, -300L, memAndDisk) - assert(status.memUsedByRdd(0) === 910L) - assert(status.memUsedByRdd(1) === 1900L) - assert(status.memUsedByRdd(2) === 2730L) - assert(status.diskUsedByRdd(0) === 920L) - assert(status.diskUsedByRdd(1) === 2000L) - assert(status.diskUsedByRdd(2) === 2780L) - assert(status.offHeapUsedByRdd(0) === 901L) - assert(status.offHeapUsedByRdd(1) === 1801L) - assert(status.offHeapUsedByRdd(2) === 2701L) - - // Negative delta so large that the RDDs are no longer persisted - status.updateRddStorageInfo(0, -10000L, -10000L, -10000L, memOnly) - status.updateRddStorageInfo(1, -20000L, -20000L, -20000L, diskOnly) - status.updateRddStorageInfo(2, -30000L, -30000L, -30000L, memAndDisk) - assert(status.memUsedByRdd(0) === 0L) - assert(status.memUsedByRdd(1) === 0L) - assert(status.memUsedByRdd(2) === 0L) - assert(status.diskUsedByRdd(0) === 0L) - assert(status.diskUsedByRdd(1) === 0L) - assert(status.diskUsedByRdd(2) === 0L) - assert(status.offHeapUsedByRdd(0) === 0L) - assert(status.offHeapUsedByRdd(1) === 0L) - assert(status.offHeapUsedByRdd(2) === 0L) + def actualMemUsed: Long = status.blocks.values.map(_.memSize).fold(0L)(_ + _) + def actualDiskUsed: Long = status.blocks.values.map(_.diskSize).fold(0L)(_ + _) + def actualOffHeapUsed: Long = status.blocks.values.map(_.tachyonSize).fold(0L)(_ + _) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + status.addBlock(TestBlockId("fire"), BlockStatus(memAndDisk, 4000L, 5000L, 6000L)) + status.addBlock(TestBlockId("wire"), BlockStatus(memAndDisk, 400L, 500L, 600L)) + status.addBlock(RDDBlockId(25, 25), BlockStatus(memAndDisk, 40L, 50L, 60L)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 4L, 5L, 6L)) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 4L, 5L, 6L)) + status.updateBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 4L, 5L, 6L)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + 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) + assert(status.offHeapUsed === actualOffHeapUsed) } // For testing StorageUtils.updateRddInfo and StorageUtils.getRddBlockLocations From 14fa1c3f25ddd8afac2e613c4a79df851c3b72e7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 1 Aug 2014 19:16:49 -0700 Subject: [PATCH 15/16] Simplify some code + update a few comments We were doing some crazy tuple matching business in StorageStatus#updateStorageInfo. We should try to avoid that for code readability. --- .../apache/spark/storage/StorageUtils.scala | 52 +++++++------------ .../org/apache/spark/ui/storage/RDDPage.scala | 2 +- .../apache/spark/storage/StorageSuite.scala | 8 +-- 3 files changed, 23 insertions(+), 39 deletions(-) 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 07cfcfec1c86..851c04c72e87 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -35,12 +35,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** * Internal representation of the blocks stored in this block manager. * - * A common consumption pattern is to access only the blocks that belong to a specific RDD. - * For this use case, we should avoid linearly scanning through all the blocks, which could - * be expensive if there are thousands of blocks on each block manager. Thus, we need to store - * RDD blocks and non-RDD blocks separately. In particular, we store RDD blocks in a map - * indexed by RDD IDs, so we can filter out the blocks of interest quickly. - + * We store RDD blocks and non-RDD blocks separately to allow quick retrievals of RDD blocks. * These collections should only be mutated through the add/update/removeBlock methods. */ private val _rddBlocks = new mutable.HashMap[Int, mutable.Map[BlockId, BlockStatus]] @@ -62,10 +57,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { private val _rddStorageInfo = new mutable.HashMap[Int, (Long, Long, Long, StorageLevel)] private var _nonRddStorageInfo: (Long, Long, Long) = (0L, 0L, 0L) - /** - * Instantiate a StorageStatus with the given initial blocks. This essentially makes a copy of - * the original blocks map such that the fate of this storage status is not tied to the source. - */ + /** 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) initialBlocks.foreach { case (bid, bstatus) => addBlock(bid, bstatus) } @@ -95,7 +87,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { } /** Add the given block to this storage status. If it already exists, overwrite it. */ - def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { + private[spark] def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { updateStorageInfo(blockId, blockStatus) blockId match { case RDDBlockId(rddId, _) => @@ -106,12 +98,12 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { } /** Update the given block in this storage status. If it doesn't already exist, add it. */ - def updateBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { + private[spark] def updateBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { addBlock(blockId, blockStatus) } /** Remove the given block from this storage status. */ - def removeBlock(blockId: BlockId): Option[BlockStatus] = { + private[spark] def removeBlock(blockId: BlockId): Option[BlockStatus] = { updateStorageInfo(blockId, BlockStatus.empty) blockId match { case RDDBlockId(rddId, _) => @@ -205,9 +197,8 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** * Update the relevant storage info, taking into account any existing status for this block. - * This is exposed for testing. */ - private[spark] def updateStorageInfo(blockId: BlockId, newBlockStatus: BlockStatus): Unit = { + private def updateStorageInfo(blockId: BlockId, newBlockStatus: BlockStatus): Unit = { val oldBlockStatus = getBlock(blockId).getOrElse(BlockStatus.empty) val changeInMem = newBlockStatus.memSize - oldBlockStatus.memSize val changeInDisk = newBlockStatus.diskSize - oldBlockStatus.diskSize @@ -215,7 +206,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { val level = newBlockStatus.storageLevel // Compute new info from old info - val oldInfo: (Long, Long, Long) = blockId match { + val (oldMem, oldDisk, oldTachyon) = blockId match { case RDDBlockId(rddId, _) => _rddStorageInfo.get(rddId) .map { case (mem, disk, tachyon, _) => (mem, disk, tachyon) } @@ -223,28 +214,21 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { case _ => _nonRddStorageInfo } - val newInfo: (Long, Long, Long) = oldInfo match { - case (oldMem, oldDisk, oldTachyon) => - val newMem = math.max(oldMem + changeInMem, 0L) - val newDisk = math.max(oldDisk + changeInDisk, 0L) - val newTachyon = math.max(oldTachyon + changeInTachyon, 0L) - (newMem, newDisk, newTachyon) - } + val newMem = math.max(oldMem + changeInMem, 0L) + val newDisk = math.max(oldDisk + changeInDisk, 0L) + val newTachyon = math.max(oldTachyon + changeInTachyon, 0L) // Set the correct info blockId match { case RDDBlockId(rddId, _) => - newInfo match { - case (mem, disk, tachyon) => - // If this RDD is no longer persisted, remove it - if (mem + disk + tachyon == 0) { - _rddStorageInfo.remove(rddId) - } else { - _rddStorageInfo(rddId) = (mem, disk, tachyon, level) - } + // If this RDD is no longer persisted, remove it + if (newMem + newDisk + newTachyon == 0) { + _rddStorageInfo.remove(rddId) + } else { + _rddStorageInfo(rddId) = (newMem, newDisk, newTachyon, level) } case _ => - _nonRddStorageInfo = newInfo + _nonRddStorageInfo = (newMem, newDisk, newTachyon) } } @@ -277,9 +261,9 @@ private[spark] object StorageUtils { } /** - * Return mapping from block ID to its locations for each block that belongs to the given RDD. + * Return a mapping from block ID to its locations for each block that belongs to the given RDD. */ - def getRddBlockLocations(statuses: Seq[StorageStatus], rddId: Int): Map[BlockId, Seq[String]] = { + def getRddBlockLocations(rddId: Int, statuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { val blockLocations = new mutable.HashMap[BlockId, mutable.ListBuffer[String]] statuses.foreach { status => status.rddBlocksById(rddId).foreach { case (bid, _) => 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 e2fea872cd4d..84ac53da4755 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 @@ -45,7 +45,7 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) // Block table - val blockLocations = StorageUtils.getRddBlockLocations(storageStatusList, rddId) + val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) val blocks = storageStatusList .flatMap(_.rddBlocksById(rddId)) .sortWith(_._1.name < _._1.name) 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 4509d7a47f50..298ceab6d616 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -310,8 +310,8 @@ class StorageSuite extends FunSuite { test("StorageUtils.getRddBlockLocations") { val storageStatuses = stockStorageStatuses - val blockLocations0 = StorageUtils.getRddBlockLocations(storageStatuses, 0) - val blockLocations1 = StorageUtils.getRddBlockLocations(storageStatuses, 1) + val blockLocations0 = StorageUtils.getRddBlockLocations(0, storageStatuses) + val blockLocations1 = StorageUtils.getRddBlockLocations(1, storageStatuses) assert(blockLocations0.size === 5) assert(blockLocations1.size === 3) assert(blockLocations0.contains(RDDBlockId(0, 0))) @@ -337,8 +337,8 @@ class StorageSuite extends FunSuite { storageStatuses(0).addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) storageStatuses(0).addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) storageStatuses(2).addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) - val blockLocations0 = StorageUtils.getRddBlockLocations(storageStatuses, 0) - val blockLocations1 = StorageUtils.getRddBlockLocations(storageStatuses, 1) + val blockLocations0 = StorageUtils.getRddBlockLocations(0, storageStatuses) + val blockLocations1 = StorageUtils.getRddBlockLocations(1, storageStatuses) assert(blockLocations0.size === 5) assert(blockLocations1.size === 3) assert(blockLocations0(RDDBlockId(0, 0)) === Seq("dog:1", "cat:3")) From f80c1fa13b915b3a43d91bfae79cee77240d6b3d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 1 Aug 2014 22:42:16 -0700 Subject: [PATCH 16/16] Rewrite fold and reduceOption as sum --- .../spark/storage/BlockManagerSource.scala | 10 +++++----- .../org/apache/spark/storage/StorageUtils.scala | 16 ++++++++-------- .../org/apache/spark/ui/exec/ExecutorsPage.scala | 6 +++--- .../org/apache/spark/storage/StorageSuite.scala | 6 +++--- 4 files changed, 19 insertions(+), 19 deletions(-) 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 760c508f3f09..e939318a029d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -30,7 +30,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + val maxMem = storageStatusList.map(_.maxMem).sum maxMem / 1024 / 1024 } }) @@ -38,7 +38,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("memory", "remainingMem_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + val remainingMem = storageStatusList.map(_.memRemaining).sum remainingMem / 1024 / 1024 } }) @@ -46,8 +46,8 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("memory", "memUsed_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + val maxMem = storageStatusList.map(_.maxMem).sum + val remainingMem = storageStatusList.map(_.memRemaining).sum (maxMem - remainingMem) / 1024 / 1024 } }) @@ -55,7 +55,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.map(_.diskUsed).reduceOption(_ + _).getOrElse(0L) + val diskSpaceUsed = storageStatusList.map(_.diskUsed).sum diskSpaceUsed / 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 851c04c72e87..0a0a448baa2e 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -159,7 +159,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { * Return the number of RDD blocks stored in this block manager in O(RDDs) time. * Note that this is much faster than `this.rddBlocks.size`, which is O(RDD blocks) time. */ - def numRddBlocks: Int = _rddBlocks.values.map(_.size).fold(0)(_ + _) + def numRddBlocks: Int = _rddBlocks.values.map(_.size).sum /** * Return the number of blocks that belong to the given RDD in O(1) time. @@ -173,15 +173,15 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** Return the memory used by this block manager. */ def memUsed: Long = - _nonRddStorageInfo._1 + _rddBlocks.keys.toSeq.map(memUsedByRdd).fold(0L)(_ + _) + _nonRddStorageInfo._1 + _rddBlocks.keys.toSeq.map(memUsedByRdd).sum /** Return the disk space used by this block manager. */ def diskUsed: Long = - _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).fold(0L)(_ + _) + _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum /** Return the off-heap space used by this block manager. */ def offHeapUsed: Long = - _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(offHeapUsedByRdd).fold(0L)(_ + _) + _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(offHeapUsedByRdd).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) @@ -247,10 +247,10 @@ private[spark] object StorageUtils { // Assume all blocks belonging to the same RDD have the same storage level val storageLevel = statuses .map(_.rddStorageLevel(rddId)).flatMap(s => s).headOption.getOrElse(StorageLevel.NONE) - val numCachedPartitions = statuses.map(_.numRddBlocksById(rddId)).fold(0)(_ + _) - val memSize = statuses.map(_.memUsedByRdd(rddId)).fold(0L)(_ + _) - val diskSize = statuses.map(_.diskUsedByRdd(rddId)).fold(0L)(_ + _) - val tachyonSize = statuses.map(_.offHeapUsedByRdd(rddId)).fold(0L)(_ + _) + val numCachedPartitions = statuses.map(_.numRddBlocksById(rddId)).sum + val memSize = statuses.map(_.memUsedByRdd(rddId)).sum + val diskSize = statuses.map(_.diskUsedByRdd(rddId)).sum + val tachyonSize = statuses.map(_.offHeapUsedByRdd(rddId)).sum rddInfo.storageLevel = storageLevel rddInfo.numCachedPartitions = numCachedPartitions 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 8b18627a17e1..b814b0e6b850 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 @@ -49,9 +49,9 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList - val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) - val memUsed = storageStatusList.map(_.memUsed).fold(0L)(_ + _) - val diskUsed = storageStatusList.map(_.diskUsed).fold(0L)(_ + _) + val maxMem = storageStatusList.map(_.maxMem).sum + val memUsed = storageStatusList.map(_.memUsed).sum + val diskUsed = storageStatusList.map(_.diskUsed).sum val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.id) 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 298ceab6d616..38678bbd1dd2 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -242,9 +242,9 @@ class StorageSuite extends FunSuite { test("storage status memUsed, diskUsed, tachyonUsed") { val status = storageStatus2 - def actualMemUsed: Long = status.blocks.values.map(_.memSize).fold(0L)(_ + _) - def actualDiskUsed: Long = status.blocks.values.map(_.diskSize).fold(0L)(_ + _) - def actualOffHeapUsed: Long = status.blocks.values.map(_.tachyonSize).fold(0L)(_ + _) + def actualMemUsed: Long = status.blocks.values.map(_.memSize).sum + def actualDiskUsed: Long = status.blocks.values.map(_.diskSize).sum + def actualOffHeapUsed: Long = status.blocks.values.map(_.tachyonSize).sum assert(status.memUsed === actualMemUsed) assert(status.diskUsed === actualDiskUsed) assert(status.offHeapUsed === actualOffHeapUsed)