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 6b9193540085e39aef94c5f0116549556216994e..0ea35e2b7a311c020132f34408dd894bb62a9a64 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() {