From 6fae936088d2a50606ba5082cee4a3c3a98a2b01 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 2 May 2013 22:30:06 -0700 Subject: applications (aka drivers) send their webUI address to master when registering so it can be displayed in the master web ui --- .../scala/spark/deploy/ApplicationDescription.scala | 4 +++- .../src/main/scala/spark/deploy/client/TestClient.scala | 2 +- .../scala/spark/deploy/master/ApplicationInfo.scala | 8 +++++++- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 3 ++- core/src/main/scala/spark/storage/BlockManagerUI.scala | 17 +++++++++-------- .../twirl/spark/deploy/master/app_details.scala.html | 1 + 7 files changed, 24 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index 6659e53b25..bb9e7b3bba 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -5,7 +5,9 @@ private[spark] class ApplicationDescription( val cores: Int, val memoryPerSlave: Int, val command: Command, - val sparkHome: String) + val sparkHome: String, + val appUIHost: String, + val appUIPort: Int) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index ad92532b58..e4ab01dd2a 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -25,7 +25,7 @@ private[spark] object TestClient { val url = args(0) val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) val desc = new ApplicationDescription( - "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home") + "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "localhost", 0) val listener = new TestListener val client = new Client(actorSystem, url, desc, listener) client.start() diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 3591a94072..3ee1b60351 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -10,7 +10,9 @@ private[spark] class ApplicationInfo( val id: String, val desc: ApplicationDescription, val submitDate: Date, - val driver: ActorRef) + val driver: ActorRef, + val appUIHost: String, + val appUIPort: Int) { var state = ApplicationState.WAITING var executors = new mutable.HashMap[Int, ExecutorInfo] @@ -60,4 +62,8 @@ private[spark] class ApplicationInfo( System.currentTimeMillis() - startTime } } + + + def appUIAddress = "http://" + this.appUIHost + ":" + this.appUIPort + } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 160afe5239..9f2d3da495 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -244,7 +244,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver) + val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUIHost, desc.appUIPort) apps += app idToApp(app.id) = app actorToApp(driver) = app diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 0b8922d139..5d7d1feb74 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -31,7 +31,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) - val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome) + val appDesc = + new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.host, sc.ui.port) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 07da572044..13158e4262 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -20,19 +20,20 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, val STATIC_RESOURCE_DIR = "spark/deploy/static" implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val host = Utils.localHostName() + val port = if (System.getProperty("spark.ui.port") != null) { + System.getProperty("spark.ui.port").toInt + } else { + // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which + // random port it bound to, so we have to try to find a local one by creating a socket. + Utils.findFreePort() + } /** Start a HTTP server to run the Web interface */ def start() { try { - val port = if (System.getProperty("spark.ui.port") != null) { - System.getProperty("spark.ui.port").toInt - } else { - // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which - // random port it bound to, so we have to try to find a local one by creating a socket. - Utils.findFreePort() - } AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, handler, "BlockManagerHTTPServer") - logInfo("Started BlockManager web UI at http://%s:%d".format(Utils.localHostName(), port)) + logInfo("Started BlockManager web UI at http://%s:%d".format(host, port)) } catch { case e: Exception => logError("Failed to create BlockManager WebUI", e) diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html index 301a7e2124..02086b476f 100644 --- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html +++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html @@ -22,6 +22,7 @@
  • Memory per Slave: @app.desc.memoryPerSlave
  • Submit Date: @app.submitDate
  • State: @app.state
  • +
  • Application Detail UI
  • -- cgit v1.2.3 From 22a5063ae45300cdd759f51877dac850008e16ee Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 5 May 2013 12:19:11 -0700 Subject: switch from separating appUI host & port to combining into just appUiUrl --- core/src/main/scala/spark/deploy/ApplicationDescription.scala | 3 +-- core/src/main/scala/spark/deploy/client/TestClient.scala | 2 +- core/src/main/scala/spark/deploy/master/ApplicationInfo.scala | 6 +----- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- .../scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala | 2 +- core/src/main/scala/spark/storage/BlockManagerUI.scala | 2 ++ core/src/main/twirl/spark/deploy/master/app_details.scala.html | 2 +- 7 files changed, 8 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index bb9e7b3bba..4aff0aedc1 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -6,8 +6,7 @@ private[spark] class ApplicationDescription( val memoryPerSlave: Int, val command: Command, val sparkHome: String, - val appUIHost: String, - val appUIPort: Int) + val appUiUrl: String) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index e4ab01dd2a..f195082808 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -25,7 +25,7 @@ private[spark] object TestClient { val url = args(0) val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) val desc = new ApplicationDescription( - "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "localhost", 0) + "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored") val listener = new TestListener val client = new Client(actorSystem, url, desc, listener) client.start() diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 3ee1b60351..e28b007e30 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -11,8 +11,7 @@ private[spark] class ApplicationInfo( val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef, - val appUIHost: String, - val appUIPort: Int) + val appUiUrl: String) { var state = ApplicationState.WAITING var executors = new mutable.HashMap[Int, ExecutorInfo] @@ -63,7 +62,4 @@ private[spark] class ApplicationInfo( } } - - def appUIAddress = "http://" + this.appUIHost + ":" + this.appUIPort - } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 9f2d3da495..6f58ad16af 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -244,7 +244,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUIHost, desc.appUIPort) + val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl) apps += app idToApp(app.id) = app actorToApp(driver) = app diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 5d7d1feb74..955ee5d806 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -32,7 +32,7 @@ private[spark] class SparkDeploySchedulerBackend( val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) val appDesc = - new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.host, sc.ui.port) + new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.appUIAddress) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 13158e4262..e02281344a 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -74,4 +74,6 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } } } + + private[spark] def appUIAddress = "http://" + host + ":" + port } diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html index 02086b476f..15eabc9834 100644 --- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html +++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html @@ -22,7 +22,7 @@
  • Memory per Slave: @app.desc.memoryPerSlave
  • Submit Date: @app.submitDate
  • State: @app.state
  • -
  • Application Detail UI
  • +
  • Application Detail UI
  • -- cgit v1.2.3 From 0ab818d50812f312596170b5e42aa76d2ff59d15 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 9 May 2013 00:38:59 -0700 Subject: fix linebreak --- .../scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 955ee5d806..170ede0f44 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -31,8 +31,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) - val appDesc = - new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.appUIAddress) + val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, + sc.ui.appUIAddress) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() -- cgit v1.2.3