blob: 7bea8ac13fd4a688ac18705415563c109e25a0c1 [file] [log] [blame]
<!DOCTYPE html>
<!--[if lt IE 7]> <html class="no-js lt-ie9 lt-ie8 lt-ie7"> <![endif]-->
<!--[if IE 7]> <html class="no-js lt-ie9 lt-ie8"> <![endif]-->
<!--[if IE 8]> <html class="no-js lt-ie9"> <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js"> <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta http-equiv="X-UA-Compatible" content="IE=edge,chrome=1">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>Spark Standalone Mode - Spark 3.5.0 Documentation</title>
<link rel="stylesheet" href="css/bootstrap.min.css">
<link rel="preconnect" href="https://fonts.googleapis.com">
<link rel="preconnect" href="https://fonts.gstatic.com" crossorigin>
<link href="https://fonts.googleapis.com/css2?family=DM+Sans:ital,wght@0,400;0,500;0,700;1,400;1,500;1,700&Courier+Prime:wght@400;700&display=swap" rel="stylesheet">
<link href="css/custom.css" rel="stylesheet">
<script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script>
<link rel="stylesheet" href="css/pygments-default.css">
<link rel="stylesheet" href="css/docsearch.min.css" />
<link rel="stylesheet" href="css/docsearch.css">
<!-- Matomo -->
<script type="text/javascript">
var _paq = window._paq = window._paq || [];
/* tracker methods like "setCustomDimension" should be called before "trackPageView" */
_paq.push(["disableCookies"]);
_paq.push(['trackPageView']);
_paq.push(['enableLinkTracking']);
(function() {
var u="https://analytics.apache.org/";
_paq.push(['setTrackerUrl', u+'matomo.php']);
_paq.push(['setSiteId', '40']);
var d=document, g=d.createElement('script'), s=d.getElementsByTagName('script')[0];
g.async=true; g.src=u+'matomo.js'; s.parentNode.insertBefore(g,s);
})();
</script>
<!-- End Matomo Code -->
</head>
<body class="global">
<!--[if lt IE 7]>
<p class="chromeframe">You are using an outdated browser. <a href="https://browsehappy.com/">Upgrade your browser today</a> or <a href="http://www.google.com/chromeframe/?redirect=true">install Google Chrome Frame</a> to better experience this site.</p>
<![endif]-->
<!-- This code is taken from http://twitter.github.com/bootstrap/examples/hero.html -->
<nav class="navbar navbar-expand-lg navbar-dark p-0 px-4 fixed-top" style="background: #1d6890;" id="topbar">
<div class="navbar-brand"><a href="index.html">
<img src="img/spark-logo-rev.svg" width="141" height="72"/></a><span class="version">3.5.0</span>
</div>
<button class="navbar-toggler" type="button" data-toggle="collapse"
data-target="#navbarCollapse" aria-controls="navbarCollapse"
aria-expanded="false" aria-label="Toggle navigation">
<span class="navbar-toggler-icon"></span>
</button>
<div class="collapse navbar-collapse" id="navbarCollapse">
<ul class="navbar-nav me-auto">
<li class="nav-item"><a href="index.html" class="nav-link">Overview</a></li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarQuickStart" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">Programming Guides</a>
<div class="dropdown-menu" aria-labelledby="navbarQuickStart">
<a class="dropdown-item" href="quick-start.html">Quick Start</a>
<a class="dropdown-item" href="rdd-programming-guide.html">RDDs, Accumulators, Broadcasts Vars</a>
<a class="dropdown-item" href="sql-programming-guide.html">SQL, DataFrames, and Datasets</a>
<a class="dropdown-item" href="structured-streaming-programming-guide.html">Structured Streaming</a>
<a class="dropdown-item" href="streaming-programming-guide.html">Spark Streaming (DStreams)</a>
<a class="dropdown-item" href="ml-guide.html">MLlib (Machine Learning)</a>
<a class="dropdown-item" href="graphx-programming-guide.html">GraphX (Graph Processing)</a>
<a class="dropdown-item" href="sparkr.html">SparkR (R on Spark)</a>
<a class="dropdown-item" href="api/python/getting_started/index.html">PySpark (Python on Spark)</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarAPIDocs" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">API Docs</a>
<div class="dropdown-menu" aria-labelledby="navbarAPIDocs">
<a class="dropdown-item" href="api/scala/org/apache/spark/index.html">Scala</a>
<a class="dropdown-item" href="api/java/index.html">Java</a>
<a class="dropdown-item" href="api/python/index.html">Python</a>
<a class="dropdown-item" href="api/R/index.html">R</a>
<a class="dropdown-item" href="api/sql/index.html">SQL, Built-in Functions</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarDeploying" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">Deploying</a>
<div class="dropdown-menu" aria-labelledby="navbarDeploying">
<a class="dropdown-item" href="cluster-overview.html">Overview</a>
<a class="dropdown-item" href="submitting-applications.html">Submitting Applications</a>
<div class="dropdown-divider"></div>
<a class="dropdown-item" href="spark-standalone.html">Spark Standalone</a>
<a class="dropdown-item" href="running-on-mesos.html">Mesos</a>
<a class="dropdown-item" href="running-on-yarn.html">YARN</a>
<a class="dropdown-item" href="running-on-kubernetes.html">Kubernetes</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarMore" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">More</a>
<div class="dropdown-menu" aria-labelledby="navbarMore">
<a class="dropdown-item" href="configuration.html">Configuration</a>
<a class="dropdown-item" href="monitoring.html">Monitoring</a>
<a class="dropdown-item" href="tuning.html">Tuning Guide</a>
<a class="dropdown-item" href="job-scheduling.html">Job Scheduling</a>
<a class="dropdown-item" href="security.html">Security</a>
<a class="dropdown-item" href="hardware-provisioning.html">Hardware Provisioning</a>
<a class="dropdown-item" href="migration-guide.html">Migration Guide</a>
<div class="dropdown-divider"></div>
<a class="dropdown-item" href="building-spark.html">Building Spark</a>
<a class="dropdown-item" href="https://spark.apache.org/contributing.html">Contributing to Spark</a>
<a class="dropdown-item" href="https://spark.apache.org/third-party-projects.html">Third Party Projects</a>
</div>
</li>
<li class="nav-item">
<input type="text" id="docsearch-input" placeholder="Search the docs…">
</li>
</ul>
<!--<span class="navbar-text navbar-right"><span class="version-text">v3.5.0</span></span>-->
</div>
</nav>
<div class="container">
<div class="content mr-3" id="content">
<h1 class="title">Spark Standalone Mode</h1>
<ul id="markdown-toc">
<li><a href="#security" id="markdown-toc-security">Security</a></li>
<li><a href="#installing-spark-standalone-to-a-cluster" id="markdown-toc-installing-spark-standalone-to-a-cluster">Installing Spark Standalone to a Cluster</a></li>
<li><a href="#starting-a-cluster-manually" id="markdown-toc-starting-a-cluster-manually">Starting a Cluster Manually</a></li>
<li><a href="#cluster-launch-scripts" id="markdown-toc-cluster-launch-scripts">Cluster Launch Scripts</a></li>
<li><a href="#resource-allocation-and-configuration-overview" id="markdown-toc-resource-allocation-and-configuration-overview">Resource Allocation and Configuration Overview</a></li>
<li><a href="#connecting-an-application-to-the-cluster" id="markdown-toc-connecting-an-application-to-the-cluster">Connecting an Application to the Cluster</a></li>
<li><a href="#client-properties" id="markdown-toc-client-properties">Client Properties</a></li>
<li><a href="#launching-spark-applications" id="markdown-toc-launching-spark-applications">Launching Spark Applications</a></li>
<li><a href="#resource-scheduling" id="markdown-toc-resource-scheduling">Resource Scheduling</a></li>
<li><a href="#executors-scheduling" id="markdown-toc-executors-scheduling">Executors Scheduling</a></li>
<li><a href="#stage-level-scheduling-overview" id="markdown-toc-stage-level-scheduling-overview">Stage Level Scheduling Overview</a> <ul>
<li><a href="#caveats" id="markdown-toc-caveats">Caveats</a></li>
</ul>
</li>
<li><a href="#monitoring-and-logging" id="markdown-toc-monitoring-and-logging">Monitoring and Logging</a></li>
<li><a href="#running-alongside-hadoop" id="markdown-toc-running-alongside-hadoop">Running Alongside Hadoop</a></li>
<li><a href="#configuring-ports-for-network-security" id="markdown-toc-configuring-ports-for-network-security">Configuring Ports for Network Security</a></li>
<li><a href="#high-availability" id="markdown-toc-high-availability">High Availability</a> <ul>
<li><a href="#standby-masters-with-zookeeper" id="markdown-toc-standby-masters-with-zookeeper">Standby Masters with ZooKeeper</a></li>
<li><a href="#single-node-recovery-with-local-file-system" id="markdown-toc-single-node-recovery-with-local-file-system">Single-Node Recovery with Local File System</a></li>
</ul>
</li>
</ul>
<p>In addition to running on the Mesos or YARN cluster managers, Spark also provides a simple standalone deploy mode. You can launch a standalone cluster either manually, by starting a master and workers by hand, or use our provided <a href="#cluster-launch-scripts">launch scripts</a>. It is also possible to run these daemons on a single machine for testing.</p>
<h1 id="security">Security</h1>
<p>Security features like authentication are not enabled by default. When deploying a cluster that is open to the internet
or an untrusted network, it&#8217;s important to secure access to the cluster to prevent unauthorized applications
from running on the cluster.
Please see <a href="security.html">Spark Security</a> and the specific security sections in this doc before running Spark.</p>
<h1 id="installing-spark-standalone-to-a-cluster">Installing Spark Standalone to a Cluster</h1>
<p>To install Spark Standalone mode, you simply place a compiled version of Spark on each node on the cluster. You can obtain pre-built versions of Spark with each release or <a href="building-spark.html">build it yourself</a>.</p>
<h1 id="starting-a-cluster-manually">Starting a Cluster Manually</h1>
<p>You can start a standalone master server by executing:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>./sbin/start-master.sh
</code></pre></div></div>
<p>Once started, the master will print out a <code class="language-plaintext highlighter-rouge">spark://HOST:PORT</code> URL for itself, which you can use to connect workers to it,
or pass as the &#8220;master&#8221; argument to <code class="language-plaintext highlighter-rouge">SparkContext</code>. You can also find this URL on
the master&#8217;s web UI, which is <a href="http://localhost:8080">http://localhost:8080</a> by default.</p>
<p>Similarly, you can start one or more workers and connect them to the master via:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>./sbin/start-worker.sh &lt;master-spark-URL&gt;
</code></pre></div></div>
<p>Once you have started a worker, look at the master&#8217;s web UI (<a href="http://localhost:8080">http://localhost:8080</a> by default).
You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS).</p>
<p>Finally, the following configuration options can be passed to the master and worker:</p>
<table class="table table-striped">
<thead><tr><th style="width:21%">Argument</th><th>Meaning</th></tr></thead>
<tr>
<td><code>-h HOST</code>, <code>--host HOST</code></td>
<td>Hostname to listen on</td>
</tr>
<tr>
<td><code>-i HOST</code>, <code>--ip HOST</code></td>
<td>Hostname to listen on (deprecated, use -h or --host)</td>
</tr>
<tr>
<td><code>-p PORT</code>, <code>--port PORT</code></td>
<td>Port for service to listen on (default: 7077 for master, random for worker)</td>
</tr>
<tr>
<td><code>--webui-port PORT</code></td>
<td>Port for web UI (default: 8080 for master, 8081 for worker)</td>
</tr>
<tr>
<td><code>-c CORES</code>, <code>--cores CORES</code></td>
<td>Total CPU cores to allow Spark applications to use on the machine (default: all available); only on worker</td>
</tr>
<tr>
<td><code>-m MEM</code>, <code>--memory MEM</code></td>
<td>Total amount of memory to allow Spark applications to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GiB); only on worker</td>
</tr>
<tr>
<td><code>-d DIR</code>, <code>--work-dir DIR</code></td>
<td>Directory to use for scratch space and job output logs (default: SPARK_HOME/work); only on worker</td>
</tr>
<tr>
<td><code>--properties-file FILE</code></td>
<td>Path to a custom Spark properties file to load (default: conf/spark-defaults.conf)</td>
</tr>
</table>
<h1 id="cluster-launch-scripts">Cluster Launch Scripts</h1>
<p>To launch a Spark standalone cluster with the launch scripts, you should create a file called conf/workers in your Spark directory,
which must contain the hostnames of all the machines where you intend to start Spark workers, one per line.
If conf/workers does not exist, the launch scripts defaults to a single machine (localhost), which is useful for testing.
Note, the master machine accesses each of the worker machines via ssh. By default, ssh is run in parallel and requires password-less (using a private key) access to be setup.
If you do not have a password-less setup, you can set the environment variable SPARK_SSH_FOREGROUND and serially provide a password for each worker.</p>
<p>Once you&#8217;ve set up this file, you can launch or stop your cluster with the following shell scripts, based on Hadoop&#8217;s deploy scripts, and available in <code class="language-plaintext highlighter-rouge">SPARK_HOME/sbin</code>:</p>
<ul>
<li><code class="language-plaintext highlighter-rouge">sbin/start-master.sh</code> - Starts a master instance on the machine the script is executed on.</li>
<li><code class="language-plaintext highlighter-rouge">sbin/start-workers.sh</code> - Starts a worker instance on each machine specified in the <code class="language-plaintext highlighter-rouge">conf/workers</code> file.</li>
<li><code class="language-plaintext highlighter-rouge">sbin/start-worker.sh</code> - Starts a worker instance on the machine the script is executed on.</li>
<li><code class="language-plaintext highlighter-rouge">sbin/start-connect-server.sh</code> - Starts a Spark Connect server on the machine the script is executed on.</li>
<li><code class="language-plaintext highlighter-rouge">sbin/start-all.sh</code> - Starts both a master and a number of workers as described above.</li>
<li><code class="language-plaintext highlighter-rouge">sbin/stop-master.sh</code> - Stops the master that was started via the <code class="language-plaintext highlighter-rouge">sbin/start-master.sh</code> script.</li>
<li><code class="language-plaintext highlighter-rouge">sbin/stop-worker.sh</code> - Stops all worker instances on the machine the script is executed on.</li>
<li><code class="language-plaintext highlighter-rouge">sbin/stop-workers.sh</code> - Stops all worker instances on the machines specified in the <code class="language-plaintext highlighter-rouge">conf/workers</code> file.</li>
<li><code class="language-plaintext highlighter-rouge">sbin/stop-connect-server.sh</code> - Stops all Spark Connect server instances on the machine the script is executed on.</li>
<li><code class="language-plaintext highlighter-rouge">sbin/stop-all.sh</code> - Stops both the master and the workers as described above.</li>
</ul>
<p>Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine.</p>
<p>You can optionally configure the cluster further by setting environment variables in <code class="language-plaintext highlighter-rouge">conf/spark-env.sh</code>. Create this file by starting with the <code class="language-plaintext highlighter-rouge">conf/spark-env.sh.template</code>, and <em>copy it to all your worker machines</em> for the settings to take effect. The following settings are available:</p>
<table class="table table-striped">
<thead><tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr></thead>
<tr>
<td><code>SPARK_MASTER_HOST</code></td>
<td>Bind the master to a specific hostname or IP address, for example a public one.</td>
</tr>
<tr>
<td><code>SPARK_MASTER_PORT</code></td>
<td>Start the master on a different port (default: 7077).</td>
</tr>
<tr>
<td><code>SPARK_MASTER_WEBUI_PORT</code></td>
<td>Port for the master web UI (default: 8080).</td>
</tr>
<tr>
<td><code>SPARK_MASTER_OPTS</code></td>
<td>Configuration properties that apply only to the master in the form "-Dx=y" (default: none). See below for a list of possible options.</td>
</tr>
<tr>
<td><code>SPARK_LOCAL_DIRS</code></td>
<td>
Directory to use for "scratch" space in Spark, including map output files and RDDs that get
stored on disk. This should be on a fast, local disk in your system. It can also be a
comma-separated list of multiple directories on different disks.
</td>
</tr>
<tr>
<td><code>SPARK_WORKER_CORES</code></td>
<td>Total number of cores to allow Spark applications to use on the machine (default: all available cores).</td>
</tr>
<tr>
<td><code>SPARK_WORKER_MEMORY</code></td>
<td>Total amount of memory to allow Spark applications to use on the machine, e.g. <code>1000m</code>, <code>2g</code> (default: total memory minus 1 GiB); note that each application's <i>individual</i> memory is configured using its <code>spark.executor.memory</code> property.</td>
</tr>
<tr>
<td><code>SPARK_WORKER_PORT</code></td>
<td>Start the Spark worker on a specific port (default: random).</td>
</tr>
<tr>
<td><code>SPARK_WORKER_WEBUI_PORT</code></td>
<td>Port for the worker web UI (default: 8081).</td>
</tr>
<tr>
<td><code>SPARK_WORKER_DIR</code></td>
<td>Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work).</td>
</tr>
<tr>
<td><code>SPARK_WORKER_OPTS</code></td>
<td>Configuration properties that apply only to the worker in the form "-Dx=y" (default: none). See below for a list of possible options.</td>
</tr>
<tr>
<td><code>SPARK_DAEMON_MEMORY</code></td>
<td>Memory to allocate to the Spark master and worker daemons themselves (default: 1g).</td>
</tr>
<tr>
<td><code>SPARK_DAEMON_JAVA_OPTS</code></td>
<td>JVM options for the Spark master and worker daemons themselves in the form "-Dx=y" (default: none).</td>
</tr>
<tr>
<td><code>SPARK_DAEMON_CLASSPATH</code></td>
<td>Classpath for the Spark master and worker daemons themselves (default: none).</td>
</tr>
<tr>
<td><code>SPARK_PUBLIC_DNS</code></td>
<td>The public DNS name of the Spark master and workers (default: none).</td>
</tr>
</table>
<p><strong>Note:</strong> The launch scripts do not currently support Windows. To run a Spark cluster on Windows, start the master and workers by hand.</p>
<p>SPARK_MASTER_OPTS supports the following system properties:</p>
<table class="table table-striped">
<thead><tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr></thead>
<tr>
<td><code>spark.deploy.retainedApplications</code></td>
<td>200</td>
<td>
The maximum number of completed applications to display. Older applications will be dropped from the UI to maintain this limit.<br />
</td>
<td>0.8.0</td>
</tr>
<tr>
<td><code>spark.deploy.retainedDrivers</code></td>
<td>200</td>
<td>
The maximum number of completed drivers to display. Older drivers will be dropped from the UI to maintain this limit.<br />
</td>
<td>1.1.0</td>
</tr>
<tr>
<td><code>spark.deploy.spreadOut</code></td>
<td>true</td>
<td>
Whether the standalone cluster manager should spread applications out across nodes or try
to consolidate them onto as few nodes as possible. Spreading out is usually better for
data locality in HDFS, but consolidating is more efficient for compute-intensive workloads. <br />
</td>
<td>0.6.1</td>
</tr>
<tr>
<td><code>spark.deploy.defaultCores</code></td>
<td>(infinite)</td>
<td>
Default number of cores to give to applications in Spark's standalone mode if they don't
set <code>spark.cores.max</code>. If not set, applications always get all available
cores unless they configure <code>spark.cores.max</code> themselves.
Set this lower on a shared cluster to prevent users from grabbing
the whole cluster by default. <br />
</td>
<td>0.9.0</td>
</tr>
<tr>
<td><code>spark.deploy.maxExecutorRetries</code></td>
<td>10</td>
<td>
Limit on the maximum number of back-to-back executor failures that can occur before the
standalone cluster manager removes a faulty application. An application will never be removed
if it has any running executors. If an application experiences more than
<code>spark.deploy.maxExecutorRetries</code> failures in a row, no executors
successfully start running in between those failures, and the application has no running
executors then the standalone cluster manager will remove the application and mark it as failed.
To disable this automatic removal, set <code>spark.deploy.maxExecutorRetries</code> to
<code>-1</code>.
<br />
</td>
<td>1.6.3</td>
</tr>
<tr>
<td><code>spark.worker.timeout</code></td>
<td>60</td>
<td>
Number of seconds after which the standalone deploy master considers a worker lost if it
receives no heartbeats.
</td>
<td>0.6.2</td>
</tr>
<tr>
<td><code>spark.worker.resource.{resourceName}.amount</code></td>
<td>(none)</td>
<td>
Amount of a particular resource to use on the worker.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.worker.resource.{resourceName}.discoveryScript</code></td>
<td>(none)</td>
<td>
Path to resource discovery script, which is used to find a particular resource while worker starting up.
And the output of the script should be formatted like the <code>ResourceInformation</code> class.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.worker.resourcesFile</code></td>
<td>(none)</td>
<td>
Path to resources file which is used to find various resources while worker starting up.
The content of resources file should be formatted like <code>
[{"id":{"componentName": "spark.worker","resourceName":"gpu"},"addresses":["0","1","2"]}]</code>.
If a particular resource is not found in the resources file, the discovery script would be used to
find that resource. If the discovery script also does not find the resources, the worker will fail
to start up.
</td>
<td>3.0.0</td>
</tr>
</table>
<p>SPARK_WORKER_OPTS supports the following system properties:</p>
<table class="table table-striped">
<thead><tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr></thead>
<tr>
<td><code>spark.worker.cleanup.enabled</code></td>
<td>false</td>
<td>
Enable periodic cleanup of worker / application directories. Note that this only affects standalone
mode, as YARN works differently. Only the directories of stopped applications are cleaned up.
This should be enabled if spark.shuffle.service.db.enabled is "true"
</td>
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.worker.cleanup.interval</code></td>
<td>1800 (30 minutes)</td>
<td>
Controls the interval, in seconds, at which the worker cleans up old application work dirs
on the local machine.
</td>
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.worker.cleanup.appDataTtl</code></td>
<td>604800 (7 days, 7 * 24 * 3600)</td>
<td>
The number of seconds to retain application work directories on each worker. This is a Time To Live
and should depend on the amount of available disk space you have. Application logs and jars are
downloaded to each application work dir. Over time, the work dirs can quickly fill up disk space,
especially if you run jobs very frequently.
</td>
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.shuffle.service.db.enabled</code></td>
<td>true</td>
<td>
Store External Shuffle service state on local disk so that when the external shuffle service is restarted, it will
automatically reload info on current executors. This only affects standalone mode (yarn always has this behavior
enabled). You should also enable <code>spark.worker.cleanup.enabled</code>, to ensure that the state
eventually gets cleaned up. This config may be removed in the future.
</td>
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.shuffle.service.db.backend</code></td>
<td>LEVELDB</td>
<td>
When <code>spark.shuffle.service.db.enabled</code> is true, user can use this to specify the kind of disk-based
store used in shuffle service state store. This supports `LEVELDB` and `ROCKSDB` now and `LEVELDB` as default value.
The original data store in `LevelDB/RocksDB` will not be automatically convert to another kind of storage now.
</td>
<td>3.4.0</td>
</tr>
<tr>
<td><code>spark.storage.cleanupFilesAfterExecutorExit</code></td>
<td>true</td>
<td>
Enable cleanup non-shuffle files(such as temp. shuffle blocks, cached RDD/broadcast blocks,
spill files, etc) of worker directories following executor exits. Note that this doesn't
overlap with `spark.worker.cleanup.enabled`, as this enables cleanup of non-shuffle files in
local directories of a dead executor, while `spark.worker.cleanup.enabled` enables cleanup of
all files/subdirectories of a stopped and timeout application.
This only affects Standalone mode, support of other cluster managers can be added in the future.
</td>
<td>2.4.0</td>
</tr>
<tr>
<td><code>spark.worker.ui.compressedLogFileLengthCacheSize</code></td>
<td>100</td>
<td>
For compressed log files, the uncompressed file can only be computed by uncompressing the files.
Spark caches the uncompressed file size of compressed log files. This property controls the cache
size.
</td>
<td>2.0.2</td>
</tr>
</table>
<h1 id="resource-allocation-and-configuration-overview">Resource Allocation and Configuration Overview</h1>
<p>Please make sure to have read the Custom Resource Scheduling and Configuration Overview section on the <a href="configuration.html">configuration page</a>. This section only talks about the Spark Standalone specific aspects of resource scheduling.</p>
<p>Spark Standalone has 2 parts, the first is configuring the resources for the Worker, the second is the resource allocation for a specific application.</p>
<p>The user must configure the Workers to have a set of resources available so that it can assign them out to Executors. The <code>spark.worker.resource.{resourceName}.amount</code> is used to control the amount of each resource the worker has allocated. The user must also specify either <code>spark.worker.resourcesFile</code> or <code>spark.worker.resource.{resourceName}.discoveryScript</code> to specify how the Worker discovers the resources its assigned. See the descriptions above for each of those to see which method works best for your setup.</p>
<p>The second part is running an application on Spark Standalone. The only special case from the standard Spark resource configs is when you are running the Driver in client mode. For a Driver in client mode, the user can specify the resources it uses via <code>spark.driver.resourcesFile</code> or <code>spark.driver.resource.{resourceName}.discoveryScript</code>. If the Driver is running on the same host as other Drivers, please make sure the resources file or discovery script only returns resources that do not conflict with other Drivers running on the same node.</p>
<p>Note, the user does not need to specify a discovery script when submitting an application as the Worker will start each Executor with the resources it allocates to it.</p>
<h1 id="connecting-an-application-to-the-cluster">Connecting an Application to the Cluster</h1>
<p>To run an application on the Spark cluster, simply pass the <code class="language-plaintext highlighter-rouge">spark://IP:PORT</code> URL of the master as to the <a href="rdd-programming-guide.html#initializing-spark"><code class="language-plaintext highlighter-rouge">SparkContext</code>
constructor</a>.</p>
<p>To run an interactive Spark shell against the cluster, run the following command:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>./bin/spark-shell --master spark://IP:PORT
</code></pre></div></div>
<p>You can also pass an option <code class="language-plaintext highlighter-rouge">--total-executor-cores &lt;numCores&gt;</code> to control the number of cores that spark-shell uses on the cluster.</p>
<h1 id="client-properties">Client Properties</h1>
<p>Spark applications supports the following configuration properties specific to standalone mode:</p>
<table class="table table-striped">
<thead><tr><th style="width:21%">Property Name</th><th>Default Value</th><th>Meaning</th><th>Since Version</th></tr></thead>
<tr>
<td><code>spark.standalone.submit.waitAppCompletion</code></td>
<td><code>false</code></td>
<td>
In standalone cluster mode, controls whether the client waits to exit until the application completes.
If set to <code>true</code>, the client process will stay alive polling the driver's status.
Otherwise, the client process will exit after submission.
</td>
<td>3.1.0</td>
</tr>
</table>
<h1 id="launching-spark-applications">Launching Spark Applications</h1>
<p>The <a href="submitting-applications.html"><code class="language-plaintext highlighter-rouge">spark-submit</code> script</a> provides the most straightforward way to
submit a compiled Spark application to the cluster. For standalone clusters, Spark currently
supports two deploy modes. In <code class="language-plaintext highlighter-rouge">client</code> mode, the driver is launched in the same process as the
client that submits the application. In <code class="language-plaintext highlighter-rouge">cluster</code> mode, however, the driver is launched from one
of the Worker processes inside the cluster, and the client process exits as soon as it fulfills
its responsibility of submitting the application without waiting for the application to finish.</p>
<p>If your application is launched through Spark submit, then the application jar is automatically
distributed to all worker nodes. For any additional jars that your application depends on, you
should specify them through the <code class="language-plaintext highlighter-rouge">--jars</code> flag using comma as a delimiter (e.g. <code class="language-plaintext highlighter-rouge">--jars jar1,jar2</code>).
To control the application&#8217;s configuration or execution environment, see
<a href="configuration.html">Spark Configuration</a>.</p>
<p>Additionally, standalone <code class="language-plaintext highlighter-rouge">cluster</code> mode supports restarting your application automatically if it
exited with non-zero exit code. To use this feature, you may pass in the <code class="language-plaintext highlighter-rouge">--supervise</code> flag to
<code class="language-plaintext highlighter-rouge">spark-submit</code> when launching your application. Then, if you wish to kill an application that is
failing repeatedly, you may do so through:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>./bin/spark-class org.apache.spark.deploy.Client kill &lt;master url&gt; &lt;driver ID&gt;
</code></pre></div></div>
<p>You can find the driver ID through the standalone Master web UI at <code class="language-plaintext highlighter-rouge">http://&lt;master url&gt;:8080</code>.</p>
<h1 id="resource-scheduling">Resource Scheduling</h1>
<p>The standalone cluster mode currently only supports a simple FIFO scheduler across applications.
However, to allow multiple concurrent users, you can control the maximum number of resources each
application will use.
By default, it will acquire <em>all</em> cores in the cluster, which only makes sense if you just run one
application at a time. You can cap the number of cores by setting <code class="language-plaintext highlighter-rouge">spark.cores.max</code> in your
<a href="configuration.html#spark-properties">SparkConf</a>. For example:</p>
<figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">val</span> <span class="nv">conf</span> <span class="k">=</span> <span class="k">new</span> <span class="nc">SparkConf</span><span class="o">()</span>
<span class="o">.</span><span class="py">setMaster</span><span class="o">(...)</span>
<span class="o">.</span><span class="py">setAppName</span><span class="o">(...)</span>
<span class="o">.</span><span class="py">set</span><span class="o">(</span><span class="s">"spark.cores.max"</span><span class="o">,</span> <span class="s">"10"</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">sc</span> <span class="k">=</span> <span class="k">new</span> <span class="nc">SparkContext</span><span class="o">(</span><span class="n">conf</span><span class="o">)</span></code></pre></figure>
<p>In addition, you can configure <code class="language-plaintext highlighter-rouge">spark.deploy.defaultCores</code> on the cluster master process to change the
default for applications that don&#8217;t set <code class="language-plaintext highlighter-rouge">spark.cores.max</code> to something less than infinite.
Do this by adding the following to <code class="language-plaintext highlighter-rouge">conf/spark-env.sh</code>:</p>
<figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="nb">export </span><span class="nv">SPARK_MASTER_OPTS</span><span class="o">=</span><span class="s2">"-Dspark.deploy.defaultCores=&lt;value&gt;"</span></code></pre></figure>
<p>This is useful on shared clusters where users might not have configured a maximum number of cores
individually.</p>
<h1 id="executors-scheduling">Executors Scheduling</h1>
<p>The number of cores assigned to each executor is configurable. When <code class="language-plaintext highlighter-rouge">spark.executor.cores</code> is
explicitly set, multiple executors from the same application may be launched on the same worker
if the worker has enough cores and memory. Otherwise, each executor grabs all the cores available
on the worker by default, in which case only one executor per application may be launched on each
worker during one single schedule iteration.</p>
<h1 id="stage-level-scheduling-overview">Stage Level Scheduling Overview</h1>
<p>Stage level scheduling is supported on Standalone:</p>
<ul>
<li>When dynamic allocation is disabled: It allows users to specify different task resource requirements at the stage level and will use the same executors requested at startup.</li>
<li>When dynamic allocation is enabled: Currently, when the Master allocates executors for one application, it will schedule based on the order of the ResourceProfile ids for multiple ResourceProfiles. The ResourceProfile with smaller id will be scheduled firstly. Normally this won’t matter as Spark finishes one stage before starting another one, the only case this might have an affect is in a job server type scenario, so its something to keep in mind. For scheduling, we will only take executor memory and executor cores from built-in executor resources and all other custom resources from a ResourceProfile, other built-in executor resources such as offHeap and memoryOverhead won&#8217;t take any effect. The base default profile will be created based on the spark configs when you submit an application. Executor memory and executor cores from the base default profile can be propagated to custom ResourceProfiles, but all other custom resources can not be propagated.</li>
</ul>
<h2 id="caveats">Caveats</h2>
<p>As mentioned in <a href="job-scheduling.html#dynamic-resource-allocation">Dynamic Resource Allocation</a>, if cores for each executor is not explicitly specified with dynamic allocation enabled, spark will possibly acquire much more executors than expected. So you are recommended to explicitly set executor cores for each resource profile when using stage level scheduling.</p>
<h1 id="monitoring-and-logging">Monitoring and Logging</h1>
<p>Spark&#8217;s standalone mode offers a web-based user interface to monitor the cluster. The master and each worker has its own web UI that shows cluster and job statistics. By default, you can access the web UI for the master at port 8080. The port can be changed either in the configuration file or via command-line options.</p>
<p>In addition, detailed log output for each job is also written to the work directory of each worker node (<code class="language-plaintext highlighter-rouge">SPARK_HOME/work</code> by default). You will see two files for each job, <code class="language-plaintext highlighter-rouge">stdout</code> and <code class="language-plaintext highlighter-rouge">stderr</code>, with all output it wrote to its console.</p>
<h1 id="running-alongside-hadoop">Running Alongside Hadoop</h1>
<p>You can run Spark alongside your existing Hadoop cluster by just launching it as a separate service on the same machines. To access Hadoop data from Spark, just use an hdfs:// URL (typically <code class="language-plaintext highlighter-rouge">hdfs://&lt;namenode&gt;:9000/path</code>, but you can find the right URL on your Hadoop Namenode&#8217;s web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on).</p>
<h1 id="configuring-ports-for-network-security">Configuring Ports for Network Security</h1>
<p>Generally speaking, a Spark cluster and its services are not deployed on the public internet.
They are generally private services, and should only be accessible within the network of the
organization that deploys Spark. Access to the hosts and ports used by Spark services should
be limited to origin hosts that need to access the services.</p>
<p>This is particularly important for clusters using the standalone resource manager, as they do
not support fine-grained access control in a way that other resource managers do.</p>
<p>For a complete list of ports to configure, see the
<a href="security.html#configuring-ports-for-network-security">security page</a>.</p>
<h1 id="high-availability">High Availability</h1>
<p>By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below.</p>
<h2 id="standby-masters-with-zookeeper">Standby Masters with ZooKeeper</h2>
<p><strong>Overview</strong></p>
<p>Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected &#8220;leader&#8221; and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master&#8217;s state, and then resume scheduling. The entire recovery process (from the time the first leader goes down) should take between 1 and 2 minutes. Note that this delay only affects scheduling <em>new</em> applications &#8211; applications that were already running during Master failover are unaffected.</p>
<p>Learn more about getting started with ZooKeeper <a href="https://zookeeper.apache.org/doc/current/zookeeperStarted.html">here</a>.</p>
<p><strong>Configuration</strong></p>
<p>In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env by configuring <code class="language-plaintext highlighter-rouge">spark.deploy.recoveryMode</code> and related spark.deploy.zookeeper.* configurations.
For more information about these configurations please refer to the <a href="configuration.html#deploy">configuration doc</a></p>
<p>Possible gotcha: If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they&#8217;re all leaders. This will not lead to a healthy cluster state (as all Masters will schedule independently).</p>
<p><strong>Details</strong></p>
<p>After you have a ZooKeeper cluster set up, enabling high availability is straightforward. Simply start multiple Master processes on different nodes with the same ZooKeeper configuration (ZooKeeper URL and directory). Masters can be added and removed at any time.</p>
<p>In order to schedule new applications or add Workers to the cluster, they need to know the IP address of the current leader. This can be accomplished by simply passing in a list of Masters where you used to pass in a single one. For example, you might start your SparkContext pointing to <code class="language-plaintext highlighter-rouge">spark://host1:port1,host2:port2</code>. This would cause your SparkContext to try registering with both Masters &#8211; if <code class="language-plaintext highlighter-rouge">host1</code> goes down, this configuration would still be correct as we&#8217;d find the new leader, <code class="language-plaintext highlighter-rouge">host2</code>.</p>
<p>There&#8217;s an important distinction to be made between &#8220;registering with a Master&#8221; and normal operation. When starting up, an application or Worker needs to be able to find and register with the current lead Master. Once it successfully registers, though, it is &#8220;in the system&#8221; (i.e., stored in ZooKeeper). If failover occurs, the new leader will contact all previously registered applications and Workers to inform them of the change in leadership, so they need not even have known of the existence of the new Master at startup.</p>
<p>Due to this property, new Masters can be created at any time, and the only thing you need to worry about is that <em>new</em> applications and Workers can find it to register with in case it becomes the leader. Once registered, you&#8217;re taken care of.</p>
<h2 id="single-node-recovery-with-local-file-system">Single-Node Recovery with Local File System</h2>
<p><strong>Overview</strong></p>
<p>ZooKeeper is the best way to go for production-level high availability, but if you just want to be able to restart the Master if it goes down, FILESYSTEM mode can take care of it. When applications and Workers register, they have enough state written to the provided directory so that they can be recovered upon a restart of the Master process.</p>
<p><strong>Configuration</strong></p>
<p>In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration:</p>
<table class="table table-striped">
<thead><tr><th style="width:21%">System property</th><th>Meaning</th><th>Since Version</th></tr></thead>
<tr>
<td><code>spark.deploy.recoveryMode</code></td>
<td>Set to FILESYSTEM to enable single-node recovery mode (default: NONE).</td>
<td>0.8.1</td>
</tr>
<tr>
<td><code>spark.deploy.recoveryDirectory</code></td>
<td>The directory in which Spark will store recovery state, accessible from the Master's perspective.</td>
<td>0.8.1</td>
</tr>
</table>
<p><strong>Details</strong></p>
<ul>
<li>This solution can be used in tandem with a process monitor/manager like <a href="https://mmonit.com/monit/">monit</a>, or just to enable manual recovery via restart.</li>
<li>While filesystem recovery seems straightforwardly better than not doing any recovery at all, this mode may be suboptimal for certain development or experimental purposes. In particular, killing a master via stop-master.sh does not clean up its recovery state, so whenever you start a new Master, it will enter recovery mode. This could increase the startup time by up to 1 minute if it needs to wait for all previously-registered Workers/clients to timeout.</li>
<li>While it&#8217;s not officially supported, you could mount an NFS directory as the recovery directory. If the original Master node dies completely, you could then start a Master on a different node, which would correctly recover all previously registered Workers/applications (equivalent to ZooKeeper recovery). Future applications will have to be able to find the new Master, however, in order to register.</li>
</ul>
</div>
<!-- /container -->
</div>
<script src="js/vendor/jquery-3.5.1.min.js"></script>
<script src="js/vendor/bootstrap.bundle.min.js"></script>
<script src="js/vendor/anchor.min.js"></script>
<script src="js/main.js"></script>
<script type="text/javascript" src="js/vendor/docsearch.min.js"></script>
<script type="text/javascript">
// DocSearch is entirely free and automated. DocSearch is built in two parts:
// 1. a crawler which we run on our own infrastructure every 24 hours. It follows every link
// in your website and extract content from every page it traverses. It then pushes this
// content to an Algolia index.
// 2. a JavaScript snippet to be inserted in your website that will bind this Algolia index
// to your search input and display its results in a dropdown UI. If you want to find more
// details on how works DocSearch, check the docs of DocSearch.
docsearch({
apiKey: 'd62f962a82bc9abb53471cb7b89da35e',
appId: 'RAI69RXRSK',
indexName: 'apache_spark',
inputSelector: '#docsearch-input',
enhancedSearchInput: true,
algoliaOptions: {
'facetFilters': ["version:3.5.0"]
},
debug: false // Set debug to true if you want to inspect the dropdown
});
</script>
<!-- MathJax Section -->
<script type="text/x-mathjax-config">
MathJax.Hub.Config({
TeX: { equationNumbers: { autoNumber: "AMS" } }
});
</script>
<script>
// Note that we load MathJax this way to work with local file (file://), HTTP and HTTPS.
// We could use "//cdn.mathjax...", but that won't support "file://".
(function(d, script) {
script = d.createElement('script');
script.type = 'text/javascript';
script.async = true;
script.onload = function(){
MathJax.Hub.Config({
tex2jax: {
inlineMath: [ ["$", "$"], ["\\\\(","\\\\)"] ],
displayMath: [ ["$$","$$"], ["\\[", "\\]"] ],
processEscapes: true,
skipTags: ['script', 'noscript', 'style', 'textarea', 'pre']
}
});
};
script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') +
'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' +
'?config=TeX-AMS-MML_HTMLorMML';
d.getElementsByTagName('head')[0].appendChild(script);
}(document));
</script>
</body>
</html>