You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by zs...@apache.org on 2016/12/23 23:38:45 UTC

spark git commit: [SPARK-18991][CORE] Change ContextCleaner.referenceBuffer to use ConcurrentHashMap to make it faster

Repository: spark
Updated Branches:
  refs/heads/master 1311448ea -> a848f0ba8


[SPARK-18991][CORE] Change ContextCleaner.referenceBuffer to use ConcurrentHashMap to make it faster

## What changes were proposed in this pull request?

The time complexity of ConcurrentHashMap's `remove` is O(1). Changing ContextCleaner.referenceBuffer's type from `ConcurrentLinkedQueue` to `ConcurrentHashMap's` will make the removal much faster.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <sh...@databricks.com>

Closes #16390 from zsxwing/SPARK-18991.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a848f0ba
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a848f0ba
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a848f0ba

Branch: refs/heads/master
Commit: a848f0ba84e37fd95d0f47863ec68326e3296b33
Parents: 1311448
Author: Shixiong Zhu <sh...@databricks.com>
Authored: Fri Dec 23 15:38:41 2016 -0800
Committer: Shixiong Zhu <sh...@databricks.com>
Committed: Fri Dec 23 15:38:41 2016 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/ContextCleaner.scala   | 18 ++++++++++++------
 1 file changed, 12 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/a848f0ba/core/src/main/scala/org/apache/spark/ContextCleaner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala
index af91345..4d884de 100644
--- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala
@@ -18,7 +18,8 @@
 package org.apache.spark
 
 import java.lang.ref.{ReferenceQueue, WeakReference}
-import java.util.concurrent.{ConcurrentLinkedQueue, ScheduledExecutorService, TimeUnit}
+import java.util.Collections
+import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue, ScheduledExecutorService, TimeUnit}
 
 import scala.collection.JavaConverters._
 
@@ -58,7 +59,12 @@ private class CleanupTaskWeakReference(
  */
 private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
 
-  private val referenceBuffer = new ConcurrentLinkedQueue[CleanupTaskWeakReference]()
+  /**
+   * A buffer to ensure that `CleanupTaskWeakReference`s are not garbage collected as long as they
+   * have not been handled by the reference queue.
+   */
+  private val referenceBuffer =
+    Collections.newSetFromMap[CleanupTaskWeakReference](new ConcurrentHashMap)
 
   private val referenceQueue = new ReferenceQueue[AnyRef]
 
@@ -176,10 +182,10 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
           .map(_.asInstanceOf[CleanupTaskWeakReference])
         // Synchronize here to avoid being interrupted on stop()
         synchronized {
-          reference.map(_.task).foreach { task =>
-            logDebug("Got cleaning task " + task)
-            referenceBuffer.remove(reference.get)
-            task match {
+          reference.foreach { ref =>
+            logDebug("Got cleaning task " + ref.task)
+            referenceBuffer.remove(ref)
+            ref.task match {
               case CleanRDD(rddId) =>
                 doCleanupRDD(rddId, blocking = blockOnCleanupTasks)
               case CleanShuffle(shuffleId) =>


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org