aboutsummaryrefslogtreecommitdiff
path: root/core
diff options
context:
space:
mode:
authorMatei Zaharia <matei@eecs.berkeley.edu>2013-11-14 19:43:55 -0800
committerMatei Zaharia <matei@eecs.berkeley.edu>2013-11-14 19:43:55 -0800
commited25105fd9733acd631dab0993560ac66ffeae16 (patch)
tree4d159b1af59798a9a71dc2bfd120c517826c2d5a /core
parent1a4cfbea334c7b0dae287eab4c3131c8f4b8a992 (diff)
parent0ea1f8b225031d5d2e44af4147ab4c8cfff4febc (diff)
downloadspark-ed25105fd9733acd631dab0993560ac66ffeae16.tar.gz
spark-ed25105fd9733acd631dab0993560ac66ffeae16.tar.bz2
spark-ed25105fd9733acd631dab0993560ac66ffeae16.zip
Merge pull request #174 from ahirreddy/master
Write Spark UI url to driver file on HDFS This makes the SIMR code path simpler
Diffstat (limited to 'core')
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala12
1 files changed, 1 insertions, 11 deletions
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 6b91935400..0ea35e2b7a 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() {