diff --git a/core/src/main/resources/spark/deploy/master/webui/index.html b/core/src/main/resources/spark/deploy/master/webui/index.html deleted file mode 100644 index c11101045ed7b098b7c6e3bea55d08687973f56d..0000000000000000000000000000000000000000 --- a/core/src/main/resources/spark/deploy/master/webui/index.html +++ /dev/null @@ -1,6 +0,0 @@ -<html> -<head><title>Hello world!</title></head> -<body> -<p>Hello world!</p> -</body> -</html> \ No newline at end of file diff --git a/core/src/main/resources/spark/deploy/master/webui/spark_logo.png b/core/src/main/resources/spark/deploy/master/webui/spark_logo.png new file mode 100644 index 0000000000000000000000000000000000000000..4b187347792a64a6aad514826227866ff5fb27a9 Binary files /dev/null and b/core/src/main/resources/spark/deploy/master/webui/spark_logo.png differ diff --git a/core/src/main/resources/spark/deploy/worker/webui/spark_logo.png b/core/src/main/resources/spark/deploy/worker/webui/spark_logo.png new file mode 100644 index 0000000000000000000000000000000000000000..4b187347792a64a6aad514826227866ff5fb27a9 Binary files /dev/null and b/core/src/main/resources/spark/deploy/worker/webui/spark_logo.png differ diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index cf5e42797bb587cd9a89c0042f3f6c39c04dfacc..e05ca623672662dc480a4422f8de789c27ed1e8e 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -1,12 +1,17 @@ package spark.deploy import spark.deploy.ExecutorState.ExecutorState +import spark.deploy.master.{WorkerInfo, JobInfo} +import spark.deploy.worker.ExecutorRunner +import scala.collection.immutable.List +import scala.collection.mutable.HashMap + sealed trait DeployMessage extends Serializable // Worker to Master -case class RegisterWorker(id: String, host: String, port: Int, cores: Int, memory: Int) +case class RegisterWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int) extends DeployMessage case class ExecutorStateChanged( @@ -44,4 +49,19 @@ case class JobKilled(message: String) // Internal message in Client -case object StopClient \ No newline at end of file +case object StopClient + +// MasterWebUI To Master + +case object RequestMasterState + +// Master to MasterWebUI + +case class MasterState(workers: List[WorkerInfo], jobs: HashMap[String, JobInfo]) + +// WorkerWebUI to Worker +case object RequestWorkerState + +// Worker to WorkerWebUI + +case class WorkerState(workerId: String, executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner], masterUrl: String, cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int) \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index d691613b0d855c3e7e420bbb5b0b3d3546ecd066..4ccf3ee9d5896846d8a88f9c6ae8764491b9dce5 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -51,13 +51,13 @@ class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { } override def receive = { - case RegisterWorker(id, host, workerPort, cores, memory) => { + case RegisterWorker(id, host, workerPort, cores, memory, webUiPort) => { logInfo("Registering worker %s:%d with %d cores, %s RAM".format( host, workerPort, cores, Utils.memoryMegabytesToString(memory))) if (idToWorker.contains(id)) { sender ! RegisterWorkerFailed("Duplicate worker ID") } else { - addWorker(id, host, workerPort, cores, memory) + addWorker(id, host, workerPort, cores, memory, webUiPort) context.watch(sender) // This doesn't work with remote actors but helps for testing sender ! RegisteredWorker schedule() @@ -112,6 +112,10 @@ class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { addressToWorker.get(address).foreach(removeWorker) addressToJob.get(address).foreach(removeJob) } + + case RequestMasterState => { + sender ! MasterState(workers.toList, idToJob.clone) + } } /** @@ -143,8 +147,8 @@ class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging { exec.job.actor ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory) } - def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int): WorkerInfo = { - val worker = new WorkerInfo(id, host, port, cores, memory, sender) + def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int): WorkerInfo = { + val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort) workers += worker idToWorker(worker.id) = worker actorToWorker(sender) = worker diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index b0c871dd7b86397a1eaa64abecd7537711c425c4..5ee4d7730db02f7a27a830a3ee390399e1779796 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -1,7 +1,14 @@ package spark.deploy.master import akka.actor.{ActorRef, ActorSystem} +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.Timeout +import akka.util.duration._ import cc.spray.Directives +import cc.spray.directives._ +import cc.spray.typeconversion.TwirlSupport._ +import spark.deploy._ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives { val RESOURCE_DIR = "spark/deploy/master/webui" @@ -9,9 +16,29 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct val handler = { get { path("") { - getFromResource(RESOURCE_DIR + "/index.html") + completeWith { + val masterState = getMasterState() + // Render the HTML + masterui.html.index.render(masterState.jobs.values.toList, masterState.workers) + } + } ~ + path("job") { + parameter("jobId") { jobId => + completeWith { + val masterState = getMasterState + masterui.html.job_details.render(masterState.jobs(jobId)) + } + } } ~ getFromResourceDirectory(RESOURCE_DIR) } } + + // Requests the current state from the Master and waits for the response + def getMasterState() : MasterState = { + implicit val timeout = Timeout(1 seconds) + val future = master ? RequestMasterState + return Await.result(future, timeout.duration).asInstanceOf[MasterState] + } + } diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala index af0be108ea2b4fb05c061d744dd7f080edadc675..59474a09452f9f775764c940f7b6f85ac83a1ba2 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala @@ -9,7 +9,8 @@ class WorkerInfo( val port: Int, val cores: Int, val memory: Int, - val actor: ActorRef) { + val actor: ActorRef, + val webUiPort: Int) { var executors = new mutable.HashMap[String, ExecutorInfo] // fullId => info @@ -32,4 +33,8 @@ class WorkerInfo( memoryUsed -= exec.memory } } + + def webUiAddress : String = { + "http://" + this.host + ":" + this.webUiPort + } } diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index ecd558546b5db238bef42280133dd34386dd5426..3e24380810502d017ee49ef7471d4524672822c9 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -14,16 +14,16 @@ import spark.deploy.ExecutorStateChanged * Manages the execution of one executor process. */ class ExecutorRunner( - jobId: String, - execId: Int, - jobDesc: JobDescription, - cores: Int, - memory: Int, - worker: ActorRef, - workerId: String, - hostname: String, - sparkHome: File, - workDir: File) + val jobId: String, + val execId: Int, + val jobDesc: JobDescription, + val cores: Int, + val memory: Int, + val worker: ActorRef, + val workerId: String, + val hostname: String, + val sparkHome: File, + val workDir: File) extends Logging { val fullId = jobId + "/" + execId diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 19ffc1e40132959ec69db58eba4c9fd030ff9314..fc496fdd9771e96b53750117c11b23d62bb9cd03 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -27,7 +27,7 @@ class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, mas var sparkHome: File = null var workDir: File = null val executors = new HashMap[String, ExecutorRunner] - val finishedExecutors = new ArrayBuffer[String] + val finishedExecutors = new HashMap[String, ExecutorRunner] var coresUsed = 0 var memoryUsed = 0 @@ -67,7 +67,7 @@ class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, mas val akkaUrl = "akka://spark@%s:%s/user/Master".format(masterHost, masterPort) try { master = context.actorFor(akkaUrl) - master ! RegisterWorker(workerId, ip, port, cores, memory) + master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { @@ -108,25 +108,34 @@ class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, mas jobId, execId, jobDesc, cores_, memory_, self, workerId, ip, sparkHome, workDir) executors(jobId + "/" + execId) = manager manager.start() + coresUsed += cores_ + memoryUsed += memory_ master ! ExecutorStateChanged(jobId, execId, ExecutorState.LOADING, None) case ExecutorStateChanged(jobId, execId, state, message) => master ! ExecutorStateChanged(jobId, execId, state, message) + val fullId = jobId + "/" + execId if (ExecutorState.isFinished(state)) { - logInfo("Executor " + jobId + "/" + execId + " finished with state " + state) - executors -= jobId + "/" + execId - finishedExecutors += jobId + "/" + execId + val executor = executors(fullId) + logInfo("Executor " + fullId + " finished with state " + state) + finishedExecutors(fullId) = executor + executors -= fullId + coresUsed -= executor.cores + memoryUsed -= executor.memory } case KillExecutor(jobId, execId) => val fullId = jobId + "/" + execId + val executor = executors(fullId) logInfo("Asked to kill executor " + fullId) - executors(jobId + "/" + execId).kill() - executors -= fullId - finishedExecutors += fullId + executor.kill() case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => masterDisconnected() + + case RequestWorkerState => { + sender ! WorkerState(workerId, executors.values.toList, finishedExecutors.values.toList, masterUrl, cores, memory, coresUsed, memoryUsed) + } } def masterDisconnected() { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index efd3822e6168cfec861d20318d69450374ffaf29..47760f463db0ec56e0f10836490803887daee795 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -1,7 +1,13 @@ package spark.deploy.worker import akka.actor.{ActorRef, ActorSystem} +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.Timeout +import akka.util.duration._ import cc.spray.Directives +import cc.spray.typeconversion.TwirlSupport._ +import spark.deploy.{WorkerState, RequestWorkerState} class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives { val RESOURCE_DIR = "spark/deploy/worker/webui" @@ -9,9 +15,24 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct val handler = { get { path("") { - getFromResource(RESOURCE_DIR + "/index.html") + completeWith{ + workerui.html.index(getWorkerState()) + } + } ~ + path("log") { + parameters("jobId", "executorId", "logType") { (jobId, executorId, logType) => + getFromFileName("work/" + jobId + "/" + executorId + "/" + logType) + } } ~ getFromResourceDirectory(RESOURCE_DIR) } } + + // Requests the current state from the Master and waits for the response + def getWorkerState() : WorkerState = { + implicit val timeout = Timeout(1 seconds) + val future = worker ? RequestWorkerState + return Await.result(future, timeout.duration).asInstanceOf[WorkerState] + } + } diff --git a/core/src/main/twirl/common/layout.scala.html b/core/src/main/twirl/common/layout.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..e6bb21969a32e78e9ba0e956216c38542cae5540 --- /dev/null +++ b/core/src/main/twirl/common/layout.scala.html @@ -0,0 +1,31 @@ +@(title: String)(content: Html) + +<!DOCTYPE html> +<html> + + <head> + <meta http-equiv="Content-type" content="text/html; charset=utf-8"> + <link rel="stylesheet" href="http://netdna.bootstrapcdn.com/twitter-bootstrap/2.0.4/css/bootstrap.min.css" type="text/css"> + <link rel="stylesheet" href="http://netdna.bootstrapcdn.com/twitter-bootstrap/2.0.4/css/bootstrap-responsive.min.css" type="text/css"> + <title>Spark WebUI</title> + </head> + + <body> + <div class="container"> + + <!-- HEADER --> + <div class="row"> + <div class="span12"> + <img src="spark_logo.png"> + <h1 style="vertical-align: bottom; margin-bottom: 10px; margin-left: 30px; display: inline-block;"> @title </h1> + </div> + </div> + + <hr/> + + @content + + </div> + </body> + +</html> \ No newline at end of file diff --git a/core/src/main/twirl/masterui/executor_row.scala.html b/core/src/main/twirl/masterui/executor_row.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..784d692fc2f91c2faa964ed76c41dfce91876e65 --- /dev/null +++ b/core/src/main/twirl/masterui/executor_row.scala.html @@ -0,0 +1,15 @@ +@(executor: spark.deploy.master.ExecutorInfo) + +<tr> + <td>@executor.id</td> + <td> + <a href="@executor.worker.webUiAddress">@executor.worker.id</href> + </td> + <td>@executor.cores</td> + <td>@executor.memory</td> + <td>@executor.state</td> + <td> + <a href="@(executor.worker.webUiAddress)/log?jobId=@(executor.job.id)&executorId=@(executor.id)&logType=stdout">stdout</a> + <a href="@(executor.worker.webUiAddress)/log?jobId=@(executor.job.id)&executorId=@(executor.id)&logType=stderr">stderr</a> + </td> +</tr> \ No newline at end of file diff --git a/core/src/main/twirl/masterui/executors_table.scala.html b/core/src/main/twirl/masterui/executors_table.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..cafc42c80e822f69cd20fe1d5480c0a95957ee5f --- /dev/null +++ b/core/src/main/twirl/masterui/executors_table.scala.html @@ -0,0 +1,19 @@ +@(executors: List[spark.deploy.master.ExecutorInfo]) + +<table class="table table-bordered table-striped table-condensed"> + <thead> + <tr> + <th>ExecutorID</th> + <th>Worker</th> + <th>Cores</th> + <th>Memory</th> + <th>State</th> + <th>Logs</th> + </tr> + </thead> + <tbody> + @for(e <- executors) { + @executor_row(e) + } + </tbody> +</table> \ No newline at end of file diff --git a/core/src/main/twirl/masterui/index.scala.html b/core/src/main/twirl/masterui/index.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..ddf61637650c0667f6e09104347a7259b5a65b53 --- /dev/null +++ b/core/src/main/twirl/masterui/index.scala.html @@ -0,0 +1,37 @@ +@(jobs: List[spark.deploy.master.JobInfo], workers: List[spark.deploy.master.WorkerInfo]) +@import spark.deploy.master._ + +@common.html.layout(title = "Master WebUI") { + + <!-- Cluster Summary (Workers) --> + <div class="row"> + <div class="span12"> + <h3> Cluster Summary </h3> + <br/> + @worker_table(workers) + </div> + </div> + + <hr/> + + <!-- Job Summary (Running) --> + <div class="row"> + <div class="span12"> + <h3> Running Jobs </h3> + <br/> + @job_table(jobs.filter(j => j.state == JobState.WAITING || j.state == JobState.RUNNING)) + </div> + </div> + + <hr/> + + <!-- Job Summary (Completed) --> + <div class="row"> + <div class="span12"> + <h3> Completed Jobs </h3> + <br/> + @job_table(jobs.filter(j => j.state == JobState.FINISHED || j.state == JobState.FAILED)) + </div> + </div> + +} \ No newline at end of file diff --git a/core/src/main/twirl/masterui/job_details.scala.html b/core/src/main/twirl/masterui/job_details.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..a1fa4ab1ac7cda331580ab4c55d97f48784ee3ef --- /dev/null +++ b/core/src/main/twirl/masterui/job_details.scala.html @@ -0,0 +1,34 @@ +@(job: spark.deploy.master.JobInfo) + +@common.html.layout(title = "Job Details") { + + <!-- Job Details --> + <div class="row"> + <div class="span12"> + <ul class="unstyled"> + <li><strong>ID:</strong> @job.id</li> + <li><strong>Description:</strong> @job.desc.name</li> + <li><strong>User:</strong> @job.desc.user</li> + <li><strong>Cores:</strong> @job.desc.cores</li> + <li><strong>Memory per Slave:</strong> @job.desc.memoryPerSlave</li> + <li><strong>Submit Date:</strong> @job.submitDate</li> + <li><strong>State:</strong> @job.state</li> + <li><strong>Cores Granted:</strong> @job.coresGranted</li> + <li><strong>Cores Left:</strong> @job.coresLeft</li> + <li><strong>Command:</strong> @job.desc.command</li> + </ul> + </div> + </div> + + <hr/> + + <!-- Executors --> + <div class="row"> + <div class="span12"> + <h3> Executor Summary </h3> + <br/> + @executors_table(job.executors.values.toList) + </div> + </div> + +} \ No newline at end of file diff --git a/core/src/main/twirl/masterui/job_row.scala.html b/core/src/main/twirl/masterui/job_row.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..1d0d1650c0f6a751b3b1e7058fc3765b4bb5dd92 --- /dev/null +++ b/core/src/main/twirl/masterui/job_row.scala.html @@ -0,0 +1,15 @@ +@(job: spark.deploy.master.JobInfo) + +<tr> + <td> + <a href="job?jobId=@(job.id)">@job.id</a> + </td> + <td>@job.desc</td> + <td> + @job.coresGranted Granted, @job.coresLeft Left + </td> + <td>@job.desc.memoryPerSlave</td> + <td>@job.submitDate</td> + <td>@job.desc.user</td> + <td>@job.state.toString()</td> +</tr> \ No newline at end of file diff --git a/core/src/main/twirl/masterui/job_table.scala.html b/core/src/main/twirl/masterui/job_table.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..b3b1e4d4725b0aa231b8f4827fa85a2f176bf41e --- /dev/null +++ b/core/src/main/twirl/masterui/job_table.scala.html @@ -0,0 +1,20 @@ +@(jobs: List[spark.deploy.master.JobInfo]) + +<table class="table table-bordered table-striped table-condensed"> + <thead> + <tr> + <th>JobID</th> + <th>Description</th> + <th>Cores</th> + <th>Memory per Slave</th> + <th>Submit Date</th> + <th>User</th> + <th>State</th> + </tr> + </thead> + <tbody> + @for(j <- jobs) { + @job_row(j) + } + </tbody> +</table> \ No newline at end of file diff --git a/core/src/main/twirl/masterui/worker_row.scala.html b/core/src/main/twirl/masterui/worker_row.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..6c8aaaae604742ee5592e375ea80e234e60e7ff9 --- /dev/null +++ b/core/src/main/twirl/masterui/worker_row.scala.html @@ -0,0 +1,11 @@ +@(worker: spark.deploy.master.WorkerInfo) + +<tr> + <td> + <a href="http://@worker.host:@worker.webUiPort">@worker.id</href> + </td> + <td>@worker.host</td> + <td>@worker.port</td> + <td>@worker.cores (@worker.coresUsed Used)</td> + <td>@worker.memory (@worker.memoryUsed Used)</td> +</tr> \ No newline at end of file diff --git a/core/src/main/twirl/masterui/worker_table.scala.html b/core/src/main/twirl/masterui/worker_table.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..201af5383ab13c7bb3e9a7e265681d08680efc5c --- /dev/null +++ b/core/src/main/twirl/masterui/worker_table.scala.html @@ -0,0 +1,18 @@ +@(workers: List[spark.deploy.master.WorkerInfo]) + +<table class="table table-bordered table-striped table-condensed"> + <thead> + <tr> + <th>ID</th> + <th>Host</th> + <th>Port</th> + <th>Cores</th> + <th>Memory</th> + </tr> + </thead> + <tbody> + @for(w <- workers) { + @worker_row(w) + } + </tbody> +</table> \ No newline at end of file diff --git a/core/src/main/twirl/workerui/executor_row.scala.html b/core/src/main/twirl/workerui/executor_row.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..db3d33d74e33d31ad694dae5a335320926cdec81 --- /dev/null +++ b/core/src/main/twirl/workerui/executor_row.scala.html @@ -0,0 +1,21 @@ +@(executor: spark.deploy.worker.ExecutorRunner) + +<tr> + <td>@executor.execId</td> + <td>@executor.cores</td> + <td>@executor.memory</td> + <td> + <ul class="unstyled"> + <li><strong>ID:</strong> @executor.jobId</li> + <li><strong>Name:</strong> @executor.jobDesc.name</li> + <li><strong>User:</strong> @executor.jobDesc.user</li> + <li><strong>Cores:</strong> @executor.jobDesc.cores </li> + <li><strong>Memory per Slave:</strong> @executor.jobDesc.memoryPerSlave</li> + <li><strong>Command:</strong> @executor.jobDesc.command</li> + </ul> + </td> + <td> + <a href="log?jobId=@(executor.jobId)&executorId=@(executor.execId)&logType=stdout">stdout</a> + <a href="log?jobId=@(executor.jobId)&executorId=@(executor.execId)&logType=stderr">stderr</a> + </td> +</tr> \ No newline at end of file diff --git a/core/src/main/twirl/workerui/executors_table.scala.html b/core/src/main/twirl/workerui/executors_table.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..c8d51cc9f47b86846a1cf55fba0386065a8c6687 --- /dev/null +++ b/core/src/main/twirl/workerui/executors_table.scala.html @@ -0,0 +1,18 @@ +@(executors: List[spark.deploy.worker.ExecutorRunner]) + +<table class="table table-bordered table-striped table-condensed"> + <thead> + <tr> + <th>ExecutorID</th> + <th>Cores</th> + <th>Memory</th> + <th>Job Details</th> + <th>Logs</th> + </tr> + </thead> + <tbody> + @for(e <- executors) { + @executor_row(e) + } + </tbody> +</table> \ No newline at end of file diff --git a/core/src/main/twirl/workerui/index.scala.html b/core/src/main/twirl/workerui/index.scala.html new file mode 100644 index 0000000000000000000000000000000000000000..a70760f25a98730434642a85af5be2e67058be56 --- /dev/null +++ b/core/src/main/twirl/workerui/index.scala.html @@ -0,0 +1,39 @@ +@(worker: spark.deploy.WorkerState) + +@common.html.layout(title = "Worker WebUI") { + + <!-- Worker Details --> + <div class="row"> + <div class="span12"> + <ul class="unstyled"> + <li><strong>ID:</strong> @worker.workerId</li> + <li><strong>Master URL:</strong> @worker.masterUrl </li> + <li><strong>Cores:</strong> @worker.cores (@worker.coresUsed Used)</li> + <li><strong>Memory:</strong> @worker.memory (@worker.memoryUsed Used)</li> + </ul> + </div> + </div> + + <hr/> + + <!-- Running Executors --> + <div class="row"> + <div class="span12"> + <h3> Running Executors </h3> + <br/> + @executors_table(worker.executors) + </div> + </div> + + <hr/> + + <!-- Finished Executors --> + <div class="row"> + <div class="span12"> + <h3> Finished Executors </h3> + <br/> + @executors_table(worker.finishedExecutors) + </div> + </div> + +} \ No newline at end of file diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 726d4907381034cc86ccfe719df1bc6474428c0a..d1445f2adef2aea304705201d140b1dc19b3113c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -2,6 +2,7 @@ import sbt._ import Keys._ import sbtassembly.Plugin._ import AssemblyKeys._ +import twirl.sbt.TwirlPlugin._ object SparkBuild extends Build { // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or @@ -69,7 +70,7 @@ object SparkBuild extends Build { "cc.spray" % "spray-can" % "1.0-M2.1", "cc.spray" % "spray-server" % "1.0-M2.1" ) - ) ++ assemblySettings ++ extraAssemblySettings + ) ++ assemblySettings ++ extraAssemblySettings ++ Seq(Twirl.settings: _*) def replSettings = sharedSettings ++ Seq( name := "spark-repl", diff --git a/project/plugins.sbt b/project/plugins.sbt index 0e2b6d49028e92cd52bba3371670366dfa7286f9..896fa4834fdb7d07eccc1a5ad99bf323ea8af436 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,9 +1,13 @@ -resolvers += Classpaths.typesafeResolver - resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns) +resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" + +resolvers += "Spray Repository" at "http://repo.spray.cc/" + addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.3") addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.0-RC1") addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.0.0") + +addSbtPlugin("cc.spray" %% "sbt-twirl" % "0.5.2")