Updated Branches: refs/heads/master 1a4cfbea3 -> ed25105fd
Write Spark UI url to driver file on HDFS Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/0ea1f8b2 Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/0ea1f8b2 Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/0ea1f8b2 Branch: refs/heads/master Commit: 0ea1f8b225031d5d2e44af4147ab4c8cfff4febc Parents: 39af914 Author: Ahir Reddy <[email protected]> Authored: Wed Nov 13 15:23:36 2013 -0800 Committer: Ahir Reddy <[email protected]> Committed: Wed Nov 13 15:23:36 2013 -0800 ---------------------------------------------------------------------- .../spark/scheduler/cluster/SimrSchedulerBackend.scala | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0ea1f8b2/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 6b91935..0ea35e2 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,10 +31,6 @@ 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() { @@ -49,23 +45,17 @@ 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) temp.writeUTF(driverUrl) temp.writeInt(maxCores) + temp.writeUTF(sc.ui.appUIAddress) temp.close() // "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() {
