Skip to content

Commit 98b732a

Browse files
committed
All of the rest of the changes.
1 parent 3940e97 commit 98b732a

10 files changed

Lines changed: 25 additions & 435 deletions

File tree

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 2 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ import scala.collection.mutable.HashMap
3333
import scala.reflect.{ClassTag, classTag}
3434
import scala.util.control.NonFatal
3535

36+
import com.google.common.collect.MapMaker
3637
import org.apache.commons.lang.SerializationUtils
3738
import org.apache.hadoop.conf.Configuration
3839
import org.apache.hadoop.fs.Path
@@ -221,7 +222,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
221222
private var _eventLogDir: Option[URI] = None
222223
private var _eventLogCodec: Option[String] = None
223224
private var _env: SparkEnv = _
224-
private var _metadataCleaner: MetadataCleaner = _
225225
private var _jobProgressListener: JobProgressListener = _
226226
private var _statusTracker: SparkStatusTracker = _
227227
private var _progressBar: Option[ConsoleProgressBar] = None
@@ -295,8 +295,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
295295
private[spark] val addedJars = HashMap[String, Long]()
296296

297297
// Keeps track of all persisted RDDs
298-
private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]]
299-
private[spark] def metadataCleaner: MetadataCleaner = _metadataCleaner
298+
private[spark] val persistentRdds = new MapMaker().weakValues().makeMap[Int, RDD[_]]().asScala
300299
private[spark] def jobProgressListener: JobProgressListener = _jobProgressListener
301300

302301
def statusTracker: SparkStatusTracker = _statusTracker
@@ -463,8 +462,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
463462
_conf.set("spark.repl.class.uri", replUri)
464463
}
465464

466-
_metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, _conf)
467-
468465
_statusTracker = new SparkStatusTracker(this)
469466

470467
_progressBar =
@@ -1721,11 +1718,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
17211718
env.metricsSystem.report()
17221719
}
17231720
}
1724-
if (metadataCleaner != null) {
1725-
Utils.tryLogNonFatalError {
1726-
metadataCleaner.cancel()
1727-
}
1728-
}
17291721
Utils.tryLogNonFatalError {
17301722
_cleaner.foreach(_.stop())
17311723
}
@@ -2193,11 +2185,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
21932185
}
21942186
}
21952187

2196-
/** Called by MetadataCleaner to clean up the persistentRdds map periodically */
2197-
private[spark] def cleanup(cleanupTime: Long) {
2198-
persistentRdds.clearOldValues(cleanupTime)
2199-
}
2200-
22012188
// In order to prevent multiple SparkContexts from being active at the same time, mark this
22022189
// context as having finished construction.
22032190
// NOTE: this must be placed at the end of the SparkContext constructor.

core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala

Lines changed: 5 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
2626
import org.apache.spark.network.netty.SparkTransportConf
2727
import org.apache.spark.serializer.Serializer
2828
import org.apache.spark.storage._
29-
import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils}
29+
import org.apache.spark.util.Utils
3030
import org.apache.spark.{Logging, SparkConf, SparkEnv}
3131

3232
/** A group of writers for a ShuffleMapTask, one writer per reducer. */
@@ -63,10 +63,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf)
6363
val completedMapTasks = new ConcurrentLinkedQueue[Int]()
6464
}
6565

66-
private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState]
67-
68-
private val metadataCleaner =
69-
new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf)
66+
private val shuffleStates = new scala.collection.mutable.HashMap[ShuffleId, ShuffleState]
7067

7168
/**
7269
* Get a ShuffleWriterGroup for the given map task, which will register it as complete
@@ -75,9 +72,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf)
7572
def forMapTask(shuffleId: Int, mapId: Int, numReducers: Int, serializer: Serializer,
7673
writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = {
7774
new ShuffleWriterGroup {
78-
shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numReducers))
79-
private val shuffleState = shuffleStates(shuffleId)
80-
75+
private val shuffleState =
76+
shuffleStates.getOrElseUpdate(shuffleId, new ShuffleState(numReducers))
8177
val openStartTime = System.nanoTime
8278
val serializerInstance = serializer.newInstance()
8379
val writers: Array[DiskBlockObjectWriter] = {
@@ -131,11 +127,5 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf)
131127
}
132128
}
133129

134-
private def cleanup(cleanupTime: Long) {
135-
shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => removeShuffleBlocks(shuffleId))
136-
}
137-
138-
override def stop() {
139-
metadataCleaner.cancel()
140-
}
130+
override def stop(): Unit = {}
141131
}

core/src/main/scala/org/apache/spark/storage/BlockManager.scala

Lines changed: 14 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,9 @@ package org.apache.spark.storage
1919

2020
import java.io._
2121
import java.nio.{ByteBuffer, MappedByteBuffer}
22+
import java.util.concurrent.ConcurrentHashMap
2223

24+
import scala.collection.JavaConverters._
2325
import scala.collection.mutable.{ArrayBuffer, HashMap}
2426
import scala.concurrent.duration._
2527
import 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.

core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala

Lines changed: 0 additions & 109 deletions
This file was deleted.

0 commit comments

Comments
 (0)