Repository: spark
Updated Branches:
  refs/heads/branch-1.6 0c67993cf -> d1855adb5


[SPARK-12784][UI] Fix Spark UI IndexOutOfBoundsException with dynamic allocation

Add `listener.synchronized` to get `storageStatusList` and `execInfo` 
atomically.

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

Closes #10728 from zsxwing/SPARK-12784.

(cherry picked from commit 501e99ef0fbd2f2165095548fe67a3447ccbfc91)
Signed-off-by: Shixiong Zhu <shixi...@databricks.com>


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

Branch: refs/heads/branch-1.6
Commit: d1855adb5eab7bf42604e949fa6c9687e91bade1
Parents: 0c67993
Author: Shixiong Zhu <shixi...@databricks.com>
Authored: Thu Jan 14 09:50:57 2016 -0800
Committer: Shixiong Zhu <shixi...@databricks.com>
Committed: Thu Jan 14 09:51:07 2016 -0800

----------------------------------------------------------------------
 .../spark/status/api/v1/ExecutorListResource.scala     | 10 +++++++---
 .../scala/org/apache/spark/ui/exec/ExecutorsPage.scala | 13 ++++++++++---
 2 files changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/d1855adb/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala
----------------------------------------------------------------------
diff --git 
a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala 
b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala
index 8ad4656..3bdba92 100644
--- 
a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala
+++ 
b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala
@@ -28,9 +28,13 @@ private[v1] class ExecutorListResource(ui: SparkUI) {
   @GET
   def executorList(): Seq[ExecutorSummary] = {
     val listener = ui.executorsListener
-    val storageStatusList = listener.storageStatusList
-    (0 until storageStatusList.size).map { statusId =>
-      ExecutorsPage.getExecInfo(listener, statusId)
+    listener.synchronized {
+      // The follow codes should be protected by `listener` to make sure no 
executors will be
+      // removed before we query their status. See SPARK-12784.
+      val storageStatusList = listener.storageStatusList
+      (0 until storageStatusList.size).map { statusId =>
+        ExecutorsPage.getExecInfo(listener, statusId)
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d1855adb/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
----------------------------------------------------------------------
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 1a29b0f..7072a15 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
@@ -52,12 +52,19 @@ private[ui] class ExecutorsPage(
   private val listener = parent.listener
 
   def render(request: HttpServletRequest): Seq[Node] = {
-    val storageStatusList = listener.storageStatusList
+    val (storageStatusList, execInfo) = listener.synchronized {
+      // The follow codes should be protected by `listener` to make sure no 
executors will be
+      // removed before we query their status. See SPARK-12784.
+      val _storageStatusList = listener.storageStatusList
+      val _execInfo = {
+        for (statusId <- 0 until _storageStatusList.size)
+          yield ExecutorsPage.getExecInfo(listener, statusId)
+      }
+      (_storageStatusList, _execInfo)
+    }
     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
-      ExecutorsPage.getExecInfo(listener, statusId)
     val execInfoSorted = execInfo.sortBy(_.id)
     val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty
 


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

Reply via email to