@@ -19,7 +19,9 @@ package org.apache.spark.storage
1919
2020import java .io ._
2121import java .nio .{ByteBuffer , MappedByteBuffer }
22+ import java .util .concurrent .ConcurrentHashMap
2223
24+ import scala .collection .JavaConverters ._
2325import scala .collection .mutable .{ArrayBuffer , HashMap }
2426import scala .concurrent .duration ._
2527import scala .concurrent .{Await , ExecutionContext , Future }
@@ -75,7 +77,7 @@ private[spark] class BlockManager(
7577
7678 val diskBlockManager = new DiskBlockManager (this , conf)
7779
78- private val blockInfo = new HashMap [BlockId , BlockInfo ]
80+ private val blockInfo = new ConcurrentHashMap [BlockId , BlockInfo ]
7981
8082 private val futureExecutionContext = ExecutionContext .fromExecutorService(
8183 ThreadUtils .newDaemonCachedThreadPool(" block-manager-future" , 128 ))
@@ -227,7 +229,7 @@ private[spark] class BlockManager(
227229 */
228230 private def reportAllBlocks (): Unit = {
229231 logInfo(s " Reporting ${blockInfo.size} blocks to the master. " )
230- for ((blockId, info) <- blockInfo) {
232+ for ((blockId, info) <- blockInfo.asScala ) {
231233 val status = getCurrentBlockStatus(blockId, info)
232234 if (! tryToReportBlockStatus(blockId, info, status)) {
233235 logError(s " Failed to report $blockId to master; giving up. " )
@@ -308,7 +310,7 @@ private[spark] class BlockManager(
308310 * NOTE: This is mainly for testing, and it doesn't fetch information from external block store.
309311 */
310312 def getStatus (blockId : BlockId ): Option [BlockStatus ] = {
311- blockInfo.get(blockId).map { info =>
313+ blockInfo.asScala. get(blockId).map { info =>
312314 val memSize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L
313315 val diskSize = if (diskStore.contains(blockId)) diskStore.getSize(blockId) else 0L
314316 // Assume that block is not in external block store
@@ -322,7 +324,7 @@ private[spark] class BlockManager(
322324 * may not know of).
323325 */
324326 def getMatchingBlockIds (filter : BlockId => Boolean ): Seq [BlockId ] = {
325- (blockInfo.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq
327+ (blockInfo.asScala. keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq
326328 }
327329
328330 /**
@@ -434,15 +436,15 @@ private[spark] class BlockManager(
434436 }
435437
436438 private def doGetLocal (blockId : BlockId , asBlockResult : Boolean ): Option [Any ] = {
437- val info = blockInfo.get(blockId).orNull
439+ val info = blockInfo.get(blockId)
438440 if (info != null ) {
439441 info.synchronized {
440442 // Double check to make sure the block is still there. There is a small chance that the
441443 // block has been removed by removeBlock (which also synchronizes on the blockInfo object).
442444 // Note that this only checks metadata tracking. If user intentionally deleted the block
443445 // on disk or from off heap storage without using removeBlock, this conditional check will
444446 // still pass but eventually we will get an exception because we can't find the block.
445- if (blockInfo.get(blockId).isEmpty) {
447+ if (blockInfo.asScala. get(blockId).isEmpty) {
446448 logWarning(s " Block $blockId had been removed " )
447449 return None
448450 }
@@ -726,7 +728,7 @@ private[spark] class BlockManager(
726728 val putBlockInfo = {
727729 val tinfo = new BlockInfo (level, tellMaster)
728730 // Do atomically !
729- val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo)
731+ val oldBlockOpt = Option ( blockInfo.putIfAbsent(blockId, tinfo) )
730732 if (oldBlockOpt.isDefined) {
731733 if (oldBlockOpt.get.waitForReady()) {
732734 logWarning(s " Block $blockId already exists on this machine; not re-adding it " )
@@ -1027,7 +1029,7 @@ private[spark] class BlockManager(
10271029 data : () => Either [Array [Any ], ByteBuffer ]): Option [BlockStatus ] = {
10281030
10291031 logInfo(s " Dropping block $blockId from memory " )
1030- val info = blockInfo.get(blockId).orNull
1032+ val info = blockInfo.get(blockId)
10311033
10321034 // If the block has not already been dropped
10331035 if (info != null ) {
@@ -1038,7 +1040,7 @@ private[spark] class BlockManager(
10381040 // If we get here, the block write failed.
10391041 logWarning(s " Block $blockId was marked as failure. Nothing to drop " )
10401042 return None
1041- } else if (blockInfo.get(blockId).isEmpty) {
1043+ } else if (blockInfo.asScala. get(blockId).isEmpty) {
10421044 logWarning(s " Block $blockId was already dropped. " )
10431045 return None
10441046 }
@@ -1090,7 +1092,7 @@ private[spark] class BlockManager(
10901092 def removeRdd (rddId : Int ): Int = {
10911093 // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks.
10921094 logInfo(s " Removing RDD $rddId" )
1093- val blocksToRemove = blockInfo.keys.flatMap(_.asRDDId).filter(_.rddId == rddId)
1095+ val blocksToRemove = blockInfo.asScala. keys.flatMap(_.asRDDId).filter(_.rddId == rddId)
10941096 blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false ) }
10951097 blocksToRemove.size
10961098 }
@@ -1100,7 +1102,7 @@ private[spark] class BlockManager(
11001102 */
11011103 def removeBroadcast (broadcastId : Long , tellMaster : Boolean ): Int = {
11021104 logDebug(s " Removing broadcast $broadcastId" )
1103- val blocksToRemove = blockInfo.keys.collect {
1105+ val blocksToRemove = blockInfo.asScala. keys.collect {
11041106 case bid @ BroadcastBlockId (`broadcastId`, _) => bid
11051107 }
11061108 blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster) }
@@ -1112,7 +1114,7 @@ private[spark] class BlockManager(
11121114 */
11131115 def removeBlock (blockId : BlockId , tellMaster : Boolean = true ): Unit = {
11141116 logDebug(s " Removing block $blockId" )
1115- val info = blockInfo.get(blockId).orNull
1117+ val info = blockInfo.get(blockId)
11161118 if (info != null ) {
11171119 info.synchronized {
11181120 // Removals are idempotent in disk store and memory store. At worst, we get a warning.
0 commit comments