aboutsummaryrefslogtreecommitdiff
path: root/core
diff options
context:
space:
mode:
authorReynold Xin <reynoldx@gmail.com>2013-07-29 17:14:44 -0700
committerReynold Xin <reynoldx@gmail.com>2013-07-29 17:14:44 -0700
commit17e62113d446c87d0313daa2ac230867591627be (patch)
treef7dcf04ec8d6bfc8f575ec9e745989ec8e145419 /core
parentc99b67440537aacba836624b0bfebc513b20f6b3 (diff)
downloadspark-17e62113d446c87d0313daa2ac230867591627be.tar.gz
spark-17e62113d446c87d0313daa2ac230867591627be.tar.bz2
spark-17e62113d446c87d0313daa2ac230867591627be.zip
Moved DeployMessage's into its own DeployMessages object.
Also renamed MasterState to MasterStateResponse and WorkerState to WorkerStateResponse for clarity.
Diffstat (limited to 'core')
-rw-r--r--core/src/main/scala/spark/deploy/DeployMessage.scala146
-rw-r--r--core/src/main/scala/spark/deploy/JsonProtocol.scala11
-rw-r--r--core/src/main/scala/spark/deploy/client/Client.scala18
-rw-r--r--core/src/main/scala/spark/deploy/master/Master.scala39
-rw-r--r--core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala11
-rw-r--r--core/src/main/scala/spark/deploy/master/ui/IndexPage.scala16
-rw-r--r--core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala10
-rw-r--r--core/src/main/scala/spark/deploy/worker/Worker.scala28
-rw-r--r--core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala23
9 files changed, 160 insertions, 142 deletions
diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala
index e1f8aff6f5..7c37a16615 100644
--- a/core/src/main/scala/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/spark/deploy/DeployMessage.scala
@@ -17,109 +17,107 @@
package spark.deploy
+import scala.collection.immutable.List
+
+import spark.Utils
import spark.deploy.ExecutorState.ExecutorState
import spark.deploy.master.{WorkerInfo, ApplicationInfo}
import spark.deploy.worker.ExecutorRunner
-import scala.collection.immutable.List
-import spark.Utils
-private[spark] sealed trait DeployMessage extends Serializable
+private[deploy] sealed trait DeployMessage extends Serializable
-// Worker to Master
+private[deploy] object DeployMessages {
-private[spark]
-case class RegisterWorker(
- id: String,
- host: String,
- port: Int,
- cores: Int,
- memory: Int,
- webUiPort: Int,
- publicAddress: String)
- extends DeployMessage {
- Utils.checkHost(host, "Required hostname")
- assert (port > 0)
-}
+ // Worker to Master
-private[spark]
-case class ExecutorStateChanged(
- appId: String,
- execId: Int,
- state: ExecutorState,
- message: Option[String],
- exitStatus: Option[Int])
- extends DeployMessage
+ case class RegisterWorker(
+ id: String,
+ host: String,
+ port: Int,
+ cores: Int,
+ memory: Int,
+ webUiPort: Int,
+ publicAddress: String)
+ extends DeployMessage {
+ Utils.checkHost(host, "Required hostname")
+ assert (port > 0)
+ }
-private[spark] case class Heartbeat(workerId: String) extends DeployMessage
+ case class ExecutorStateChanged(
+ appId: String,
+ execId: Int,
+ state: ExecutorState,
+ message: Option[String],
+ exitStatus: Option[Int])
+ extends DeployMessage
-// Master to Worker
+ case class Heartbeat(workerId: String) extends DeployMessage
-private[spark] case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage
-private[spark] case class RegisterWorkerFailed(message: String) extends DeployMessage
-private[spark] case class KillExecutor(appId: String, execId: Int) extends DeployMessage
+ // Master to Worker
-private[spark] case class LaunchExecutor(
- appId: String,
- execId: Int,
- appDesc: ApplicationDescription,
- cores: Int,
- memory: Int,
- sparkHome: String)
- extends DeployMessage
+ case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage
-// Client to Master
+ case class RegisterWorkerFailed(message: String) extends DeployMessage
-private[spark] case class RegisterApplication(appDescription: ApplicationDescription)
- extends DeployMessage
+ case class KillExecutor(appId: String, execId: Int) extends DeployMessage
-// Master to Client
+ case class LaunchExecutor(
+ appId: String,
+ execId: Int,
+ appDesc: ApplicationDescription,
+ cores: Int,
+ memory: Int,
+ sparkHome: String)
+ extends DeployMessage
-private[spark]
-case class RegisteredApplication(appId: String) extends DeployMessage
+ // Client to Master
-private[spark]
-case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) {
- Utils.checkHostPort(hostPort, "Required hostport")
-}
+ case class RegisterApplication(appDescription: ApplicationDescription)
+ extends DeployMessage
-private[spark]
-case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String],
- exitStatus: Option[Int])
+ // Master to Client
-private[spark]
-case class ApplicationRemoved(message: String)
+ case class RegisteredApplication(appId: String) extends DeployMessage
-// Internal message in Client
+ case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) {
+ Utils.checkHostPort(hostPort, "Required hostport")
+ }
-private[spark] case object StopClient
+ case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String],
+ exitStatus: Option[Int])
-// MasterWebUI To Master
+ case class ApplicationRemoved(message: String)
-private[spark] case object RequestMasterState
+ // Internal message in Client
-// Master to MasterWebUI
+ case object StopClient
-private[spark]
-case class MasterState(host: String, port: Int, workers: Array[WorkerInfo],
- activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) {
+ // MasterWebUI To Master
- Utils.checkHost(host, "Required hostname")
- assert (port > 0)
+ case object RequestMasterState
- def uri = "spark://" + host + ":" + port
-}
+ // Master to MasterWebUI
+
+ case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo],
+ activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) {
+
+ Utils.checkHost(host, "Required hostname")
+ assert (port > 0)
+
+ def uri = "spark://" + host + ":" + port
+ }
-// WorkerWebUI to Worker
-private[spark] case object RequestWorkerState
+ // WorkerWebUI to Worker
+ case object RequestWorkerState
-// Worker to WorkerWebUI
+ // Worker to WorkerWebUI
-private[spark]
-case class WorkerState(host: String, port: Int, workerId: String, executors: List[ExecutorRunner],
- finishedExecutors: List[ExecutorRunner], masterUrl: String, cores: Int, memory: Int,
- coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) {
+ case class WorkerStateResponse(host: String, port: Int, workerId: String,
+ executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner], masterUrl: String,
+ cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) {
- Utils.checkHost(host, "Required hostname")
- assert (port > 0)
+ Utils.checkHost(host, "Required hostname")
+ assert (port > 0)
+ }
}
diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala
index 64f89623e1..bd1db7c294 100644
--- a/core/src/main/scala/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala
@@ -17,9 +17,12 @@
package spark.deploy
-import master.{ApplicationInfo, WorkerInfo}
import net.liftweb.json.JsonDSL._
-import worker.ExecutorRunner
+
+import spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
+import spark.deploy.master.{ApplicationInfo, WorkerInfo}
+import spark.deploy.worker.ExecutorRunner
+
private[spark] object JsonProtocol {
def writeWorkerInfo(obj: WorkerInfo) = {
@@ -57,7 +60,7 @@ private[spark] object JsonProtocol {
("appdesc" -> writeApplicationDescription(obj.appDesc))
}
- def writeMasterState(obj: MasterState) = {
+ def writeMasterState(obj: MasterStateResponse) = {
("url" -> ("spark://" + obj.uri)) ~
("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~
("cores" -> obj.workers.map(_.cores).sum) ~
@@ -68,7 +71,7 @@ private[spark] object JsonProtocol {
("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo))
}
- def writeWorkerState(obj: WorkerState) = {
+ def writeWorkerState(obj: WorkerStateResponse) = {
("id" -> obj.workerId) ~
("masterurl" -> obj.masterUrl) ~
("masterwebuiurl" -> obj.masterWebUiUrl) ~
diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala
index 29e494f495..9d5ba8a796 100644
--- a/core/src/main/scala/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/spark/deploy/client/Client.scala
@@ -17,21 +17,23 @@
package spark.deploy.client
-import spark.deploy._
+import java.util.concurrent.TimeoutException
+
import akka.actor._
+import akka.actor.Terminated
import akka.pattern.ask
import akka.util.Duration
-import akka.util.duration._
-import java.util.concurrent.TimeoutException
-import spark.{SparkException, Logging}
+import akka.remote.RemoteClientDisconnected
import akka.remote.RemoteClientLifeCycleEvent
import akka.remote.RemoteClientShutdown
-import spark.deploy.RegisterApplication
-import spark.deploy.master.Master
-import akka.remote.RemoteClientDisconnected
-import akka.actor.Terminated
import akka.dispatch.Await
+import spark.Logging
+import spark.deploy.{ApplicationDescription, ExecutorState}
+import spark.deploy.DeployMessages._
+import spark.deploy.master.Master
+
+
/**
* The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description,
* and a listener for cluster events, and calls back the listener when various events occur.
diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala
index 9692af5295..202d5bcdb7 100644
--- a/core/src/main/scala/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/spark/deploy/master/Master.scala
@@ -17,21 +17,22 @@
package spark.deploy.master
-import akka.actor._
-import akka.actor.Terminated
-import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown}
-import akka.util.duration._
-
import java.text.SimpleDateFormat
import java.util.Date
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
-import spark.deploy._
+import akka.actor._
+import akka.actor.Terminated
+import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown}
+import akka.util.duration._
+
import spark.{Logging, SparkException, Utils}
+import spark.deploy.{ApplicationDescription, ExecutorState}
+import spark.deploy.DeployMessages._
+import spark.deploy.master.ui.MasterWebUI
import spark.metrics.MetricsSystem
import spark.util.AkkaUtils
-import ui.MasterWebUI
private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
@@ -168,7 +169,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
}
case RequestMasterState => {
- sender ! MasterState(host, port, workers.toArray, apps.toArray, completedApps.toArray)
+ sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray)
}
}
@@ -233,20 +234,27 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) {
logInfo("Launching executor " + exec.fullId + " on worker " + worker.id)
worker.addExecutor(exec)
- worker.actor ! LaunchExecutor(exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome)
- exec.application.driver ! ExecutorAdded(exec.id, worker.id, worker.hostPort, exec.cores, exec.memory)
+ worker.actor ! LaunchExecutor(
+ exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome)
+ exec.application.driver ! ExecutorAdded(
+ exec.id, worker.id, worker.hostPort, exec.cores, exec.memory)
}
def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int,
publicAddress: String): WorkerInfo = {
- // There may be one or more refs to dead workers on this same node (w/ different ID's), remove them.
- workers.filter(w => (w.host == host && w.port == port) && (w.state == WorkerState.DEAD)).foreach(workers -= _)
+ // There may be one or more refs to dead workers on this same node (w/ different ID's),
+ // remove them.
+ workers.filter { w =>
+ (w.host == host && w.port == port) && (w.state == WorkerState.DEAD)
+ }.foreach { w =>
+ workers -= w
+ }
val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress)
workers += worker
idToWorker(worker.id) = worker
actorToWorker(sender) = worker
addressToWorker(sender.path.address) = worker
- return worker
+ worker
}
def removeWorker(worker: WorkerInfo) {
@@ -257,7 +265,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
addressToWorker -= worker.actor.path.address
for (exec <- worker.executors.values) {
logInfo("Telling app of lost executor: " + exec.id)
- exec.application.driver ! ExecutorUpdated(exec.id, ExecutorState.LOST, Some("worker lost"), None)
+ exec.application.driver ! ExecutorUpdated(
+ exec.id, ExecutorState.LOST, Some("worker lost"), None)
exec.application.removeExecutor(exec)
}
}
@@ -277,7 +286,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= desc.memoryPerSlave)) {
logWarning("Could not find any workers with enough memory for " + firstApp.get.id)
}
- return app
+ app
}
def finishApplication(app: ApplicationInfo) {
diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala
index 32264af393..b4c62bc224 100644
--- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala
+++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala
@@ -17,6 +17,8 @@
package spark.deploy.master.ui
+import scala.xml.Node
+
import akka.dispatch.Await
import akka.pattern.ask
import akka.util.duration._
@@ -25,9 +27,8 @@ import javax.servlet.http.HttpServletRequest
import net.liftweb.json.JsonAST.JValue
-import scala.xml.Node
-
-import spark.deploy.{RequestMasterState, JsonProtocol, MasterState}
+import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
+import spark.deploy.JsonProtocol
import spark.deploy.master.ExecutorInfo
import spark.ui.UIUtils
@@ -38,7 +39,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) {
/** Executor details for a particular application */
def renderJson(request: HttpServletRequest): JValue = {
val appId = request.getParameter("appId")
- val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState]
+ val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
val state = Await.result(stateFuture, 30 seconds)
val app = state.activeApps.find(_.id == appId).getOrElse({
state.completedApps.find(_.id == appId).getOrElse(null)
@@ -49,7 +50,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) {
/** Executor details for a particular application */
def render(request: HttpServletRequest): Seq[Node] = {
val appId = request.getParameter("appId")
- val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState]
+ val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
val state = Await.result(stateFuture, 30 seconds)
val app = state.activeApps.find(_.id == appId).getOrElse({
state.completedApps.find(_.id == appId).getOrElse(null)
diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
index b05197c1b9..557df89b41 100644
--- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
@@ -17,18 +17,20 @@
package spark.deploy.master.ui
-import akka.dispatch.Await
-import akka.pattern.ask
-import akka.util.duration._
-
import javax.servlet.http.HttpServletRequest
import scala.xml.Node
-import spark.deploy.{RequestMasterState, DeployWebUI, MasterState}
+import akka.dispatch.Await
+import akka.pattern.ask
+import akka.util.duration._
+
import spark.Utils
-import spark.ui.UIUtils
+import spark.deploy.DeployWebUI
+import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
import spark.deploy.master.{ApplicationInfo, WorkerInfo}
+import spark.ui.UIUtils
+
private[spark] class IndexPage(parent: MasterWebUI) {
val master = parent.master
@@ -36,7 +38,7 @@ private[spark] class IndexPage(parent: MasterWebUI) {
/** Index view listing applications and executors */
def render(request: HttpServletRequest): Seq[Node] = {
- val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState]
+ val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
val state = Await.result(stateFuture, 30 seconds)
val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory")
diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
index 47d3390928..345dfe879c 100644
--- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
@@ -19,14 +19,12 @@ package spark.deploy.worker
import java.io._
import java.lang.System.getenv
-import spark.deploy.{ExecutorState, ExecutorStateChanged, ApplicationDescription}
+
import akka.actor.ActorRef
+
import spark.{Utils, Logging}
-import java.net.{URI, URL}
-import org.apache.hadoop.fs.{Path, FileSystem}
-import org.apache.hadoop.conf.Configuration
-import scala.Some
-import spark.deploy.ExecutorStateChanged
+import spark.deploy.{ExecutorState, ApplicationDescription}
+import spark.deploy.DeployMessages.ExecutorStateChanged
/**
* Manages the execution of one executor process.
diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala
index 8fa0d12b82..0e46fa281e 100644
--- a/core/src/main/scala/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/spark/deploy/worker/Worker.scala
@@ -17,22 +17,24 @@
package spark.deploy.worker
-import scala.collection.mutable.{ArrayBuffer, HashMap}
+import java.text.SimpleDateFormat
+import java.util.Date
+import java.io.File
+
+import scala.collection.mutable.HashMap
+
import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
+import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
import akka.util.duration._
+
import spark.{Logging, Utils}
-import spark.util.AkkaUtils
-import spark.deploy._
-import spark.metrics.MetricsSystem
-import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
-import java.text.SimpleDateFormat
-import java.util.Date
-import spark.deploy.RegisterWorker
-import spark.deploy.LaunchExecutor
-import spark.deploy.RegisterWorkerFailed
+import spark.deploy.ExecutorState
+import spark.deploy.DeployMessages._
import spark.deploy.master.Master
-import java.io.File
-import ui.WorkerWebUI
+import spark.deploy.worker.ui.WorkerWebUI
+import spark.metrics.MetricsSystem
+import spark.util.AkkaUtils
+
private[spark] class Worker(
host: String,
@@ -164,7 +166,7 @@ private[spark] class Worker(
masterDisconnected()
case RequestWorkerState => {
- sender ! WorkerState(host, port, workerId, executors.values.toList,
+ sender ! WorkerStateResponse(host, port, workerId, executors.values.toList,
finishedExecutors.values.toList, masterUrl, cores, memory,
coresUsed, memoryUsed, masterWebUiUrl)
}
diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala
index 7548a26c2e..1619c6a4c2 100644
--- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala
@@ -17,34 +17,36 @@
package spark.deploy.worker.ui
+import javax.servlet.http.HttpServletRequest
+
+import scala.xml.Node
+
import akka.dispatch.Await
import akka.pattern.ask
import akka.util.duration._
-import javax.servlet.http.HttpServletRequest
-
import net.liftweb.json.JsonAST.JValue
-import scala.xml.Node
-
-import spark.deploy.{RequestWorkerState, JsonProtocol, WorkerState}
-import spark.deploy.worker.ExecutorRunner
import spark.Utils
+import spark.deploy.JsonProtocol
+import spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse}
+import spark.deploy.worker.ExecutorRunner
import spark.ui.UIUtils
+
private[spark] class IndexPage(parent: WorkerWebUI) {
val workerActor = parent.worker.self
val worker = parent.worker
val timeout = parent.timeout
def renderJson(request: HttpServletRequest): JValue = {
- val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState]
+ val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse]
val workerState = Await.result(stateFuture, 30 seconds)
JsonProtocol.writeWorkerState(workerState)
}
def render(request: HttpServletRequest): Seq[Node] = {
- val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState]
+ val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse]
val workerState = Await.result(stateFuture, 30 seconds)
val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs")
@@ -69,7 +71,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
<p><a href={workerState.masterWebUiUrl}>Back to Master</a></p>
</div>
</div>
- <hr/>
+ <hr/>
<div class="row"> <!-- Running Executors -->
<div class="span12">
@@ -88,7 +90,8 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
</div>
</div>;
- UIUtils.basicSparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port))
+ UIUtils.basicSparkPage(content, "Spark Worker on %s:%s".format(
+ workerState.host, workerState.port))
}
def executorRow(executor: ExecutorRunner): Seq[Node] = {