-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-7689] Remove TTL-based metadata cleaning in Spark 2.0 #10534
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 5 commits
9427635
23669a7
f2c2f5d
3940e97
98b732a
5ffe30f
e6482fa
1d96791
08b026e
75ac218
8b7ca1c
9704bc2
2451963
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,6 +33,7 @@ import scala.collection.mutable.HashMap | |
| import scala.reflect.{ClassTag, classTag} | ||
| import scala.util.control.NonFatal | ||
|
|
||
| import com.google.common.collect.MapMaker | ||
| import org.apache.commons.lang.SerializationUtils | ||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
|
|
@@ -221,7 +222,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli | |
| private var _eventLogDir: Option[URI] = None | ||
| private var _eventLogCodec: Option[String] = None | ||
| private var _env: SparkEnv = _ | ||
| private var _metadataCleaner: MetadataCleaner = _ | ||
| private var _jobProgressListener: JobProgressListener = _ | ||
| private var _statusTracker: SparkStatusTracker = _ | ||
| private var _progressBar: Option[ConsoleProgressBar] = None | ||
|
|
@@ -295,8 +295,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli | |
| private[spark] val addedJars = HashMap[String, Long]() | ||
|
|
||
| // Keeps track of all persisted RDDs | ||
| private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]] | ||
| private[spark] def metadataCleaner: MetadataCleaner = _metadataCleaner | ||
| private[spark] val persistentRdds = new MapMaker().weakValues().makeMap[Int, RDD[_]]().asScala | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'll add a comment here to clarify that MapMaker returns a ConcurrentHashMap (or will just use an explicit return type, if Guava supports that). |
||
| private[spark] def jobProgressListener: JobProgressListener = _jobProgressListener | ||
|
|
||
| def statusTracker: SparkStatusTracker = _statusTracker | ||
|
|
@@ -463,8 +462,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli | |
| _conf.set("spark.repl.class.uri", replUri) | ||
| } | ||
|
|
||
| _metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, _conf) | ||
|
|
||
| _statusTracker = new SparkStatusTracker(this) | ||
|
|
||
| _progressBar = | ||
|
|
@@ -1721,11 +1718,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli | |
| env.metricsSystem.report() | ||
| } | ||
| } | ||
| if (metadataCleaner != null) { | ||
| Utils.tryLogNonFatalError { | ||
| metadataCleaner.cancel() | ||
| } | ||
| } | ||
| Utils.tryLogNonFatalError { | ||
| _cleaner.foreach(_.stop()) | ||
| } | ||
|
|
@@ -2193,11 +2185,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli | |
| } | ||
| } | ||
|
|
||
| /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ | ||
| private[spark] def cleanup(cleanupTime: Long) { | ||
| persistentRdds.clearOldValues(cleanupTime) | ||
| } | ||
|
|
||
| // In order to prevent multiple SparkContexts from being active at the same time, mark this | ||
| // context as having finished construction. | ||
| // NOTE: this must be placed at the end of the SparkContext constructor. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,7 +26,7 @@ import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} | |
| import org.apache.spark.network.netty.SparkTransportConf | ||
| import org.apache.spark.serializer.Serializer | ||
| import org.apache.spark.storage._ | ||
| import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} | ||
| import org.apache.spark.util.Utils | ||
| import org.apache.spark.{Logging, SparkConf, SparkEnv} | ||
|
|
||
| /** A group of writers for a ShuffleMapTask, one writer per reducer. */ | ||
|
|
@@ -63,10 +63,7 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) | |
| val completedMapTasks = new ConcurrentLinkedQueue[Int]() | ||
| } | ||
|
|
||
| private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState] | ||
|
|
||
| private val metadataCleaner = | ||
| new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf) | ||
| private val shuffleStates = new scala.collection.mutable.HashMap[ShuffleId, ShuffleState] | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'll also make this into a ConcurrentHashMap. |
||
|
|
||
| /** | ||
| * Get a ShuffleWriterGroup for the given map task, which will register it as complete | ||
|
|
@@ -75,9 +72,8 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) | |
| def forMapTask(shuffleId: Int, mapId: Int, numReducers: Int, serializer: Serializer, | ||
| writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { | ||
| new ShuffleWriterGroup { | ||
| shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numReducers)) | ||
| private val shuffleState = shuffleStates(shuffleId) | ||
|
|
||
| private val shuffleState = | ||
| shuffleStates.getOrElseUpdate(shuffleId, new ShuffleState(numReducers)) | ||
| val openStartTime = System.nanoTime | ||
| val serializerInstance = serializer.newInstance() | ||
| val writers: Array[DiskBlockObjectWriter] = { | ||
|
|
@@ -131,11 +127,5 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) | |
| } | ||
| } | ||
|
|
||
| private def cleanup(cleanupTime: Long) { | ||
| shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => removeShuffleBlocks(shuffleId)) | ||
| } | ||
|
|
||
| override def stop() { | ||
| metadataCleaner.cancel() | ||
| } | ||
| override def stop(): Unit = {} | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think that these need to be ConcurrentHashMaps in order to preserve the old code's thread-safety guarantees.