Updated Branches:
  refs/heads/master f49ea28d2 -> 39af914b2

SIMR Backend Scheduler will now write Spark UI URL to HDFS, which is to be 
retrieved by SIMR clients


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

Branch: refs/heads/master
Commit: ccb099e8046947894a56b754b62d806b767f4079
Parents: dfd1ebc
Author: Ahir Reddy <[email protected]>
Authored: Tue Nov 12 15:10:47 2013 -0800
Committer: Ahir Reddy <[email protected]>
Committed: Tue Nov 12 15:58:41 2013 -0800

----------------------------------------------------------------------
 .../spark/scheduler/cluster/SimrSchedulerBackend.scala  | 12 ++++++++++++
 1 file changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ccb099e8/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
----------------------------------------------------------------------
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
index d78bdba..6b91935 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
@@ -31,6 +31,10 @@ private[spark] class SimrSchedulerBackend(
   val tmpPath = new Path(driverFilePath + "_tmp")
   val filePath = new Path(driverFilePath)
 
+  val uiFilePath = driverFilePath + "_ui"
+  val tmpUiPath = new Path(uiFilePath + "_tmp")
+  val uiPath = new Path(uiFilePath)
+
   val maxCores = System.getProperty("spark.simr.executor.cores", "1").toInt
 
   override def start() {
@@ -45,6 +49,8 @@ private[spark] class SimrSchedulerBackend(
 
     logInfo("Writing to HDFS file: "  + driverFilePath)
     logInfo("Writing Akka address: "  + driverUrl)
+    logInfo("Writing to HDFS file: "  + uiFilePath)
+    logInfo("Writing Spark UI Address: " + sc.ui.appUIAddress)
 
     // Create temporary file to prevent race condition where executors get 
empty driverUrl file
     val temp = fs.create(tmpPath, true)
@@ -54,6 +60,12 @@ private[spark] class SimrSchedulerBackend(
 
     // "Atomic" rename
     fs.rename(tmpPath, filePath)
+
+    // Write Spark UI Address to file
+    val uiTemp = fs.create(tmpUiPath, true)
+    uiTemp.writeUTF(sc.ui.appUIAddress)
+    uiTemp.close()
+    fs.rename(tmpUiPath, uiPath)
   }
 
   override def stop() {

Reply via email to