From 7a295fee9641e7b9480dd5cb520afe45039ffbe0 Mon Sep 17 00:00:00 2001 From: Denny Date: Tue, 31 Jul 2012 08:39:24 -0700 Subject: Spark WebUI Implementation. --- .../resources/spark/deploy/master/webui/index.html | 6 ---- .../spark/deploy/master/webui/spark_logo.png | Bin 0 -> 14233 bytes .../spark/deploy/worker/webui/spark_logo.png | Bin 0 -> 14233 bytes .../main/scala/spark/deploy/DeployMessage.scala | 24 +++++++++++-- .../main/scala/spark/deploy/master/Master.scala | 12 ++++--- .../scala/spark/deploy/master/MasterWebUI.scala | 29 ++++++++++++++- .../scala/spark/deploy/master/WorkerInfo.scala | 7 +++- .../scala/spark/deploy/worker/ExecutorRunner.scala | 20 +++++------ .../main/scala/spark/deploy/worker/Worker.scala | 25 ++++++++----- .../scala/spark/deploy/worker/WorkerWebUI.scala | 23 +++++++++++- core/src/main/twirl/common/layout.scala.html | 31 ++++++++++++++++ .../main/twirl/masterui/executor_row.scala.html | 15 ++++++++ .../main/twirl/masterui/executors_table.scala.html | 19 ++++++++++ core/src/main/twirl/masterui/index.scala.html | 37 +++++++++++++++++++ .../src/main/twirl/masterui/job_details.scala.html | 34 ++++++++++++++++++ core/src/main/twirl/masterui/job_row.scala.html | 15 ++++++++ core/src/main/twirl/masterui/job_table.scala.html | 20 +++++++++++ core/src/main/twirl/masterui/worker_row.scala.html | 11 ++++++ .../main/twirl/masterui/worker_table.scala.html | 18 ++++++++++ .../main/twirl/workerui/executor_row.scala.html | 21 +++++++++++ .../main/twirl/workerui/executors_table.scala.html | 18 ++++++++++ core/src/main/twirl/workerui/index.scala.html | 39 +++++++++++++++++++++ project/SparkBuild.scala | 3 +- project/plugins.sbt | 8 +++-- 24 files changed, 399 insertions(+), 36 deletions(-) delete mode 100644 core/src/main/resources/spark/deploy/master/webui/index.html create mode 100644 core/src/main/resources/spark/deploy/master/webui/spark_logo.png create mode 100644 core/src/main/resources/spark/deploy/worker/webui/spark_logo.png create mode 100644 core/src/main/twirl/common/layout.scala.html create mode 100644 core/src/main/twirl/masterui/executor_row.scala.html create mode 100644 core/src/main/twirl/masterui/executors_table.scala.html create mode 100644 core/src/main/twirl/masterui/index.scala.html create mode 100644 core/src/main/twirl/masterui/job_details.scala.html create mode 100644 core/src/main/twirl/masterui/job_row.scala.html create mode 100644 core/src/main/twirl/masterui/job_table.scala.html create mode 100644 core/src/main/twirl/masterui/worker_row.scala.html create mode 100644 core/src/main/twirl/masterui/worker_table.scala.html create mode 100644 core/src/main/twirl/workerui/executor_row.scala.html create mode 100644 core/src/main/twirl/workerui/executors_table.scala.html create mode 100644 core/src/main/twirl/workerui/index.scala.html 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 c11101045e..0000000000 --- a/core/src/main/resources/spark/deploy/master/webui/index.html +++ /dev/null @@ -1,6 +0,0 @@ - -Hello world! - -

Hello world!

- - \ 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 0000000000..4b18734779 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 0000000000..4b18734779 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 cf5e42797b..e05ca62367 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 d691613b0d..4ccf3ee9d5 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 b0c871dd7b..5ee4d7730d 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 af0be108ea..59474a0945 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 ecd558546b..3e24380810 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 19ffc1e401..fc496fdd97 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 efd3822e61..47760f463d 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 0000000000..e6bb21969a --- /dev/null +++ b/core/src/main/twirl/common/layout.scala.html @@ -0,0 +1,31 @@ +@(title: String)(content: Html) + + + + + + + + + Spark WebUI + + + +
+ + +
+
+ +

@title

+
+
+ +
+ + @content + +
+ + + \ 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 0000000000..784d692fc2 --- /dev/null +++ b/core/src/main/twirl/masterui/executor_row.scala.html @@ -0,0 +1,15 @@ +@(executor: spark.deploy.master.ExecutorInfo) + + + @executor.id + + @executor.worker.id + + @executor.cores + @executor.memory + @executor.state + + stdout + stderr + + \ 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 0000000000..cafc42c80e --- /dev/null +++ b/core/src/main/twirl/masterui/executors_table.scala.html @@ -0,0 +1,19 @@ +@(executors: List[spark.deploy.master.ExecutorInfo]) + + + + + + + + + + + + + + @for(e <- executors) { + @executor_row(e) + } + +
ExecutorIDWorkerCoresMemoryStateLogs
\ 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 0000000000..ddf6163765 --- /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

+
+ @worker_table(workers) +
+
+ +
+ + +
+
+

Running Jobs

+
+ @job_table(jobs.filter(j => j.state == JobState.WAITING || j.state == JobState.RUNNING)) +
+
+ +
+ + +
+
+

Completed Jobs

+
+ @job_table(jobs.filter(j => j.state == JobState.FINISHED || j.state == JobState.FAILED)) +
+
+ +} \ 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 0000000000..a1fa4ab1ac --- /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") { + + +
+
+
    +
  • ID: @job.id
  • +
  • Description: @job.desc.name
  • +
  • User: @job.desc.user
  • +
  • Cores: @job.desc.cores
  • +
  • Memory per Slave: @job.desc.memoryPerSlave
  • +
  • Submit Date: @job.submitDate
  • +
  • State: @job.state
  • +
  • Cores Granted: @job.coresGranted
  • +
  • Cores Left: @job.coresLeft
  • +
  • Command: @job.desc.command
  • +
+
+
+ +
+ + +
+
+

Executor Summary

+
+ @executors_table(job.executors.values.toList) +
+
+ +} \ 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 0000000000..1d0d1650c0 --- /dev/null +++ b/core/src/main/twirl/masterui/job_row.scala.html @@ -0,0 +1,15 @@ +@(job: spark.deploy.master.JobInfo) + + + + @job.id + + @job.desc + + @job.coresGranted Granted, @job.coresLeft Left + + @job.desc.memoryPerSlave + @job.submitDate + @job.desc.user + @job.state.toString() + \ 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 0000000000..b3b1e4d472 --- /dev/null +++ b/core/src/main/twirl/masterui/job_table.scala.html @@ -0,0 +1,20 @@ +@(jobs: List[spark.deploy.master.JobInfo]) + + + + + + + + + + + + + + + @for(j <- jobs) { + @job_row(j) + } + +
JobIDDescriptionCoresMemory per SlaveSubmit DateUserState
\ 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 0000000000..6c8aaaae60 --- /dev/null +++ b/core/src/main/twirl/masterui/worker_row.scala.html @@ -0,0 +1,11 @@ +@(worker: spark.deploy.master.WorkerInfo) + + + + @worker.id + + @worker.host + @worker.port + @worker.cores (@worker.coresUsed Used) + @worker.memory (@worker.memoryUsed Used) + \ 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 0000000000..201af5383a --- /dev/null +++ b/core/src/main/twirl/masterui/worker_table.scala.html @@ -0,0 +1,18 @@ +@(workers: List[spark.deploy.master.WorkerInfo]) + + + + + + + + + + + + + @for(w <- workers) { + @worker_row(w) + } + +
IDHostPortCoresMemory
\ 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 0000000000..db3d33d74e --- /dev/null +++ b/core/src/main/twirl/workerui/executor_row.scala.html @@ -0,0 +1,21 @@ +@(executor: spark.deploy.worker.ExecutorRunner) + + + @executor.execId + @executor.cores + @executor.memory + + + + +
stdout + stderr + + \ 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 0000000000..c8d51cc9f4 --- /dev/null +++ b/core/src/main/twirl/workerui/executors_table.scala.html @@ -0,0 +1,18 @@ +@(executors: List[spark.deploy.worker.ExecutorRunner]) + + + + + + + + + + + + + @for(e <- executors) { + @executor_row(e) + } + +
ExecutorIDCoresMemoryJob DetailsLogs
\ 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 0000000000..a70760f25a --- /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") { + + +
+
+
    +
  • ID: @worker.workerId
  • +
  • Master URL: @worker.masterUrl
  • +
  • Cores: @worker.cores (@worker.coresUsed Used)
  • +
  • Memory: @worker.memory (@worker.memoryUsed Used)
  • +
+
+
+ +
+ + +
+
+

Running Executors

+
+ @executors_table(worker.executors) +
+
+ +
+ + +
+
+

Finished Executors

+
+ @executors_table(worker.finishedExecutors) +
+
+ +} \ No newline at end of file diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 726d490738..d1445f2ade 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 0e2b6d4902..896fa4834f 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") -- cgit v1.2.3