Skip to content
Snippets Groups Projects
Commit ea05fc13 authored by Matei Zaharia's avatar Matei Zaharia
Browse files

Updates to standalone cluster, web UI and deploy docs.

parent 1ef4f0fb
No related branches found
No related tags found
No related merge requests found
Showing
with 53 additions and 31 deletions
......@@ -51,7 +51,7 @@ class MasterArguments(args: Array[String]) {
*/
def printUsageAndExit(exitCode: Int) {
System.err.println(
"Usage: spark-master [options]\n" +
"Usage: Master [options]\n" +
"\n" +
"Options:\n" +
" -i IP, --ip IP IP address or DNS name to listen on\n" +
......
......@@ -22,7 +22,7 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct
completeWith {
val future = master ? RequestMasterState
future.map {
masterState => masterui.html.index.render(masterState.asInstanceOf[MasterState])
masterState => spark.deploy.master.html.index.render(masterState.asInstanceOf[MasterState])
}
}
} ~
......@@ -36,7 +36,7 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct
// A bit ugly an inefficient, but we won't have a number of jobs
// so large that it will make a significant difference.
(masterState.activeJobs ::: masterState.completedJobs).find(_.id == jobId) match {
case Some(job) => masterui.html.job_details.render(job)
case Some(job) => spark.deploy.master.html.job_details.render(job)
case _ => null
}
}
......
......@@ -16,7 +16,14 @@ import spark.deploy.RegisterWorkerFailed
import akka.actor.Terminated
import java.io.File
class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, masterUrl: String)
class Worker(
ip: String,
port: Int,
webUiPort: Int,
cores: Int,
memory: Int,
masterUrl: String,
workDirPath: String = null)
extends Actor with Logging {
val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs
......@@ -37,7 +44,11 @@ class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, mas
def memoryFree: Int = memory - memoryUsed
def createWorkDir() {
workDir = new File(sparkHome, "work")
workDir = if (workDirPath != null) {
new File(workDirPath)
} else {
new File(sparkHome, "work")
}
try {
if (!workDir.exists() && !workDir.mkdirs()) {
logError("Failed to create work directory " + workDir)
......@@ -164,7 +175,8 @@ object Worker {
val args = new WorkerArguments(argStrings)
val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", args.ip, args.port)
val actor = actorSystem.actorOf(
Props(new Worker(args.ip, boundPort, args.webUiPort, args.cores, args.memory, args.master)),
Props(new Worker(args.ip, boundPort, args.webUiPort, args.cores, args.memory,
args.master, args.workDir)),
name = "Worker")
actorSystem.awaitTermination()
}
......
......@@ -15,6 +15,7 @@ class WorkerArguments(args: Array[String]) {
var cores = inferDefaultCores()
var memory = inferDefaultMemory()
var master: String = null
var workDir: String = null
// Check for settings in environment variables
if (System.getenv("SPARK_WORKER_PORT") != null) {
......@@ -29,6 +30,9 @@ class WorkerArguments(args: Array[String]) {
if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) {
webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt
}
if (System.getenv("SPARK_WORKER_DIR") != null) {
workDir = System.getenv("SPARK_WORKER_DIR")
}
parse(args.toList)
......@@ -49,6 +53,10 @@ class WorkerArguments(args: Array[String]) {
memory = value
parse(tail)
case ("--work-dir" | "-d") :: value :: tail =>
workDir = value
parse(tail)
case "--webui-port" :: IntParam(value) :: tail =>
webUiPort = value
parse(tail)
......@@ -77,13 +85,14 @@ class WorkerArguments(args: Array[String]) {
*/
def printUsageAndExit(exitCode: Int) {
System.err.println(
"Usage: spark-worker [options] <master>\n" +
"Usage: Worker [options] <master>\n" +
"\n" +
"Master must be a URL of the form spark://hostname:port\n" +
"\n" +
"Options:\n" +
" -c CORES, --cores CORES Number of cores to use\n" +
" -m MEM, --memory MEM Amount of memory to use (e.g. 1000M, 2G)\n" +
" -d DIR, --work-dir DIR Directory to run jobs in (default: SPARK_HOME/work)\n" +
" -i IP, --ip IP IP address or DNS name to listen on\n" +
" -p PORT, --port PORT Port to listen on (default: random)\n" +
" --webui-port PORT Port for web UI (default: 8081)")
......
......@@ -21,7 +21,7 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct
completeWith{
val future = worker ? RequestWorkerState
future.map { workerState =>
workerui.html.index(workerState.asInstanceOf[WorkerState])
spark.deploy.worker.html.index(workerState.asInstanceOf[WorkerState])
}
}
} ~
......
@(state: spark.deploy.MasterState)
@import spark.deploy.master._
@common.html.layout(title = "Spark Master on " + state.uri) {
@spark.deploy.common.html.layout(title = "Spark Master on " + state.uri) {
<!-- Cluster Details -->
<div class="row">
<div class="span12">
<ul class="unstyled">
<li><strong>URI:</strong> spark://@(state.uri)</li>
<li><strong>URL:</strong> spark://@(state.uri)</li>
<li><strong>Number of Workers:</strong> @state.workers.size </li>
<li><strong>Cores:</strong> @state.workers.map(_.cores).sum Total, @state.workers.map(_.coresUsed).sum Used</li>
<li><strong>Memory:</strong> @state.workers.map(_.memory).sum Total, @state.workers.map(_.memoryUsed).sum Used</li>
......@@ -47,4 +47,4 @@
</div>
</div>
}
\ No newline at end of file
}
@(job: spark.deploy.master.JobInfo)
@common.html.layout(title = "Job Details") {
@spark.deploy.common.html.layout(title = "Job Details") {
<!-- Job Details -->
<div class="row">
......@@ -37,4 +37,4 @@
</div>
</div>
}
\ No newline at end of file
}
......@@ -4,7 +4,8 @@
<td>
<a href="http://@worker.host:@worker.webUiPort">@worker.id</href>
</td>
<td>@worker.host:@worker.port</td>
<td>@{worker.host}:@{worker.port}</td>
<td>@worker.cores (@worker.coresUsed Used)</td>
<td>@worker.memory (@worker.memoryUsed Used)</td>
</tr>
\ No newline at end of file
<td>@{spark.Utils.memoryMegabytesToString(worker.memory)}
(@{spark.Utils.memoryMegabytesToString(worker.memoryUsed)} Used)</td>
</tr>
@(worker: spark.deploy.WorkerState)
@common.html.layout(title = "Spark Worker on " + worker.uri) {
@spark.deploy.common.html.layout(title = "Spark Worker on " + worker.uri) {
<!-- Worker Details -->
<div class="row">
......@@ -12,7 +12,8 @@
(WebUI at <a href="@worker.masterWebUiUrl">@worker.masterWebUiUrl</a>)
</li>
<li><strong>Cores:</strong> @worker.cores (@worker.coresUsed Used)</li>
<li><strong>Memory:</strong> @worker.memory (@worker.memoryUsed Used)</li>
<li><strong>Memory:</strong> @{spark.Utils.memoryMegabytesToString(worker.memory)}
(@{spark.Utils.memoryMegabytesToString(worker.memoryUsed)} Used)</li>
</ul>
</div>
</div>
......@@ -39,4 +40,4 @@
</div>
</div>
}
\ No newline at end of file
}
......@@ -63,7 +63,7 @@
<a href="{{HOME_PATH}}api.html" class="dropdown-toggle" data-toggle="dropdown">More<b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="configuration.html">Configuration</a></li>
<li><a href="tuning.html">Tuning</a></li>
<li><a href="tuning.html">Tuning Guide</a></li>
<li><a href="bagel-programming-guide.html">Bagel (Pregel on Spark)</a></li>
<li><a href="contributing-to-spark.html">Contributing to Spark</a></li>
</ul>
......
......@@ -80,9 +80,9 @@ there are at least four properties that you will commonly want to control:
<td>spark.cores.max</td>
<td>(infinite)</td>
<td>
When running on a <a href="{{BASE_PATH}}spark-standalone.html">standalone deploy cluster</a> or a
<a href="{{BASE_PATH}}running-on-mesos.html">Mesos cluster in "coarse-grained" sharing mode</a>,
how many CPU cores to request at most. The default will use all available cores.
When running on a <a href="{{HOME_PATH}}spark-standalone.html">standalone deploy cluster</a> or a
<a href="{{HOME_PATH}}running-on-mesos.html#mesos-run-modes">Mesos cluster in "coarse-grained"
sharing mode</a>, how many CPU cores to request at most. The default will use all available cores.
</td>
</tr>
</table>
......@@ -97,7 +97,7 @@ Apart from these, the following properties are also available, and may be useful
<td>false</td>
<td>
If set to "true", runs over Mesos clusters in
<a href="{{BASE_PATH}}running-on-mesos.html">"coarse-grained" sharing mode</a>,
<a href="{{HOME_PATH}}running-on-mesos.html#mesos-run-modes">"coarse-grained" sharing mode</a>,
where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per Spark task.
This gives lower-latency scheduling for short queries, but leaves resources in use for the whole
duration of the Spark job.
......
---
layout: global
title: Using the Spark EC2 Scripts
title: Running Spark on EC2
---
This guide describes how to get Spark running on an EC2 cluster, including how to launch clusters, how to run jobs on them, and how to shut them down. It assumes you have already signed up for Amazon EC2 account on the [Amazon Web Services site](http://aws.amazon.com/).
The `spark-ec2` script, located in Spark's `ec2` directory, allows you
to launch, manage and shut down Spark clusters on Amazon EC2. It builds
on the [Mesos EC2 script](https://github.com/mesos/mesos/wiki/EC2-Scripts)
in Apache Mesos.
to launch, manage and shut down Spark clusters on Amazon EC2. It automatically sets up Mesos, Spark and HDFS
on the cluster for you.
This guide describes how to use `spark-ec2` to launch clusters, how to run jobs on them, and how to shut them down.
It assumes you've already signed up for an EC2 account on the [Amazon Web Services site](http://aws.amazon.com/).
`spark-ec2` is designed to manage multiple named clusters. You can
launch a new cluster (telling the script its size and giving it a name),
......@@ -18,8 +18,7 @@ are derived from the name of the cluster. For example, a cluster named
`test-master`, and a number of slave nodes in a security group called
`test-slaves`. The `spark-ec2` script will create these security groups
for you based on the cluster name you request. You can also use them to
identify machines belonging to each cluster in the EC2 Console or
ElasticFox.
identify machines belonging to each cluster in the Amazon EC2 Console.
# Before You Start
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment