2015-05-01 01:24:31 -04:00
|
|
|
--
|
2012-09-03 02:05:40 -04:00
|
|
|
layout: global
|
2015-02-05 14:12:50 -05:00
|
|
|
displayTitle: Spark Configuration
|
|
|
|
title: Configuration
|
2012-09-03 02:05:40 -04:00
|
|
|
---
|
2014-05-25 20:15:47 -04:00
|
|
|
* This will become a table of contents (this text will be scraped).
|
|
|
|
{:toc}
|
2012-09-26 22:17:58 -04:00
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
Spark provides three locations to configure the system:
|
|
|
|
|
[SPARK-1566] consolidate programming guide, and general doc updates
This is a fairly large PR to clean up and update the docs for 1.0. The major changes are:
* A unified programming guide for all languages replaces language-specific ones and shows language-specific info in tabs
* New programming guide sections on key-value pairs, unit testing, input formats beyond text, migrating from 0.9, and passing functions to Spark
* Spark-submit guide moved to a separate page and expanded slightly
* Various cleanups of the menu system, security docs, and others
* Updated look of title bar to differentiate the docs from previous Spark versions
You can find the updated docs at http://people.apache.org/~matei/1.0-docs/_site/ and in particular http://people.apache.org/~matei/1.0-docs/_site/programming-guide.html.
Author: Matei Zaharia <matei@databricks.com>
Closes #896 from mateiz/1.0-docs and squashes the following commits:
03e6853 [Matei Zaharia] Some tweaks to configuration and YARN docs
0779508 [Matei Zaharia] tweak
ef671d4 [Matei Zaharia] Keep frames in JavaDoc links, and other small tweaks
1bf4112 [Matei Zaharia] Review comments
4414f88 [Matei Zaharia] tweaks
d04e979 [Matei Zaharia] Fix some old links to Java guide
a34ed33 [Matei Zaharia] tweak
541bb3b [Matei Zaharia] miscellaneous changes
fcefdec [Matei Zaharia] Moved submitting apps to separate doc
61d72b4 [Matei Zaharia] stuff
181f217 [Matei Zaharia] migration guide, remove old language guides
e11a0da [Matei Zaharia] Add more API functions
6a030a9 [Matei Zaharia] tweaks
8db0ae3 [Matei Zaharia] Added key-value pairs section
318d2c9 [Matei Zaharia] tweaks
1c81477 [Matei Zaharia] New section on basics and function syntax
e38f559 [Matei Zaharia] Actually added programming guide to Git
a33d6fe [Matei Zaharia] First pass at updating programming guide to support all languages, plus other tweaks throughout
3b6a876 [Matei Zaharia] More CSS tweaks
01ec8bf [Matei Zaharia] More CSS tweaks
e6d252e [Matei Zaharia] Change color of doc title bar to differentiate from 0.9.0
2014-05-30 03:34:33 -04:00
|
|
|
* [Spark properties](#spark-properties) control most application parameters and can be set by using
|
2014-12-11 09:21:23 -05:00
|
|
|
a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object, or through Java
|
2014-05-28 18:49:54 -04:00
|
|
|
system properties.
|
|
|
|
* [Environment variables](#environment-variables) can be used to set per-machine settings, such as
|
|
|
|
the IP address, through the `conf/spark-env.sh` script on each node.
|
|
|
|
* [Logging](#configuring-logging) can be configured through `log4j.properties`.
|
2012-09-26 22:17:58 -04:00
|
|
|
|
2013-12-30 22:17:28 -05:00
|
|
|
# Spark Properties
|
2012-09-26 22:17:58 -04:00
|
|
|
|
2014-05-12 22:44:14 -04:00
|
|
|
Spark properties control most application settings and are configured separately for each
|
2014-05-28 18:49:54 -04:00
|
|
|
application. These properties can be set directly on a
|
[SPARK-1566] consolidate programming guide, and general doc updates
This is a fairly large PR to clean up and update the docs for 1.0. The major changes are:
* A unified programming guide for all languages replaces language-specific ones and shows language-specific info in tabs
* New programming guide sections on key-value pairs, unit testing, input formats beyond text, migrating from 0.9, and passing functions to Spark
* Spark-submit guide moved to a separate page and expanded slightly
* Various cleanups of the menu system, security docs, and others
* Updated look of title bar to differentiate the docs from previous Spark versions
You can find the updated docs at http://people.apache.org/~matei/1.0-docs/_site/ and in particular http://people.apache.org/~matei/1.0-docs/_site/programming-guide.html.
Author: Matei Zaharia <matei@databricks.com>
Closes #896 from mateiz/1.0-docs and squashes the following commits:
03e6853 [Matei Zaharia] Some tweaks to configuration and YARN docs
0779508 [Matei Zaharia] tweak
ef671d4 [Matei Zaharia] Keep frames in JavaDoc links, and other small tweaks
1bf4112 [Matei Zaharia] Review comments
4414f88 [Matei Zaharia] tweaks
d04e979 [Matei Zaharia] Fix some old links to Java guide
a34ed33 [Matei Zaharia] tweak
541bb3b [Matei Zaharia] miscellaneous changes
fcefdec [Matei Zaharia] Moved submitting apps to separate doc
61d72b4 [Matei Zaharia] stuff
181f217 [Matei Zaharia] migration guide, remove old language guides
e11a0da [Matei Zaharia] Add more API functions
6a030a9 [Matei Zaharia] tweaks
8db0ae3 [Matei Zaharia] Added key-value pairs section
318d2c9 [Matei Zaharia] tweaks
1c81477 [Matei Zaharia] New section on basics and function syntax
e38f559 [Matei Zaharia] Actually added programming guide to Git
a33d6fe [Matei Zaharia] First pass at updating programming guide to support all languages, plus other tweaks throughout
3b6a876 [Matei Zaharia] More CSS tweaks
01ec8bf [Matei Zaharia] More CSS tweaks
e6d252e [Matei Zaharia] Change color of doc title bar to differentiate from 0.9.0
2014-05-30 03:34:33 -04:00
|
|
|
[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) passed to your
|
|
|
|
`SparkContext`. `SparkConf` allows you to configure some of the common properties
|
2014-05-28 18:49:54 -04:00
|
|
|
(e.g. master URL and application name), as well as arbitrary key-value pairs through the
|
2014-11-05 18:45:34 -05:00
|
|
|
`set()` method. For example, we could initialize an application with two threads as follows:
|
|
|
|
|
2014-12-11 09:21:23 -05:00
|
|
|
Note that we run with local[2], meaning two threads - which represents "minimal" parallelism,
|
|
|
|
which can help detect bugs that only exist when we run in a distributed context.
|
2012-09-26 22:17:58 -04:00
|
|
|
|
|
|
|
{% highlight scala %}
|
2014-05-21 04:23:34 -04:00
|
|
|
val conf = new SparkConf()
|
2014-11-05 18:45:34 -05:00
|
|
|
.setMaster("local[2]")
|
2014-05-12 22:44:14 -04:00
|
|
|
.setAppName("CountingSheep")
|
|
|
|
.set("spark.executor.memory", "1g")
|
2013-12-30 22:17:28 -05:00
|
|
|
val sc = new SparkContext(conf)
|
2012-09-26 22:17:58 -04:00
|
|
|
{% endhighlight %}
|
|
|
|
|
2015-04-13 19:28:07 -04:00
|
|
|
Note that we can have more than 1 thread in local mode, and in cases like Spark Streaming, we may
|
|
|
|
actually require one to prevent any sort of starvation issues.
|
2014-11-05 18:45:34 -05:00
|
|
|
|
2015-04-13 19:28:07 -04:00
|
|
|
Properties that specify some time duration should be configured with a unit of time.
|
|
|
|
The following format is accepted:
|
|
|
|
|
|
|
|
25ms (milliseconds)
|
|
|
|
5s (seconds)
|
|
|
|
10m or 10min (minutes)
|
|
|
|
3h (hours)
|
|
|
|
5d (days)
|
|
|
|
1y (years)
|
|
|
|
|
2015-04-28 15:18:55 -04:00
|
|
|
|
|
|
|
Properties that specify a byte size should be configured with a unit of size.
|
|
|
|
The following format is accepted:
|
|
|
|
|
|
|
|
1b (bytes)
|
|
|
|
1k or 1kb (kibibytes = 1024 bytes)
|
|
|
|
1m or 1mb (mebibytes = 1024 kibibytes)
|
|
|
|
1g or 1gb (gibibytes = 1024 mebibytes)
|
|
|
|
1t or 1tb (tebibytes = 1024 gibibytes)
|
|
|
|
1p or 1pb (pebibytes = 1024 tebibytes)
|
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
## Dynamically Loading Spark Properties
|
|
|
|
In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For
|
|
|
|
instance, if you'd like to run the same application with different masters or different
|
|
|
|
amounts of memory. Spark allows you to simply create an empty conf:
|
2014-05-12 22:44:14 -04:00
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
{% highlight scala %}
|
|
|
|
val sc = new SparkContext(new SparkConf())
|
|
|
|
{% endhighlight %}
|
|
|
|
|
|
|
|
Then, you can supply configuration values at runtime:
|
|
|
|
{% highlight bash %}
|
2014-12-11 09:21:23 -05:00
|
|
|
./bin/spark-submit --name "My app" --master local[4] --conf spark.shuffle.spill=false
|
|
|
|
--conf "spark.executor.extraJavaOptions=-XX:+PrintGCDetails -XX:+PrintGCTimeStamps" myApp.jar
|
2014-05-28 18:49:54 -04:00
|
|
|
{% endhighlight %}
|
|
|
|
|
2014-11-14 09:09:42 -05:00
|
|
|
The Spark shell and [`spark-submit`](submitting-applications.html)
|
2014-05-28 18:49:54 -04:00
|
|
|
tool support two ways to load configurations dynamically. The first are command line options,
|
2014-07-24 02:09:25 -04:00
|
|
|
such as `--master`, as shown above. `spark-submit` can accept any Spark property using the `--conf`
|
|
|
|
flag, but uses special flags for properties that play a part in launching the Spark application.
|
|
|
|
Running `./bin/spark-submit --help` will show the entire list of these options.
|
2014-05-28 18:49:54 -04:00
|
|
|
|
|
|
|
`bin/spark-submit` will also read configuration options from `conf/spark-defaults.conf`, in which
|
|
|
|
each line consists of a key and a value separated by whitespace. For example:
|
2014-05-12 22:44:14 -04:00
|
|
|
|
|
|
|
spark.master spark://5.6.7.8:7077
|
|
|
|
spark.executor.memory 512m
|
|
|
|
spark.eventLog.enabled true
|
|
|
|
spark.serializer org.apache.spark.serializer.KryoSerializer
|
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
Any values specified as flags or in the properties file will be passed on to the application
|
|
|
|
and merged with those specified through SparkConf. Properties set directly on the SparkConf
|
|
|
|
take highest precedence, then flags passed to `spark-submit` or `spark-shell`, then options
|
2015-03-02 19:36:42 -05:00
|
|
|
in the `spark-defaults.conf` file. A few configuration keys have been renamed since earlier
|
|
|
|
versions of Spark; in such cases, the older key names are still accepted, but take lower
|
|
|
|
precedence than any instance of the newer key.
|
2014-05-12 22:44:14 -04:00
|
|
|
|
2014-05-21 21:49:12 -04:00
|
|
|
## Viewing Spark Properties
|
|
|
|
|
|
|
|
The application web UI at `http://<driver>:4040` lists Spark properties in the "Environment" tab.
|
|
|
|
This is a useful place to check to make sure that your properties have been set correctly. Note
|
2014-12-15 17:52:17 -05:00
|
|
|
that only values explicitly specified through `spark-defaults.conf`, `SparkConf`, or the command
|
|
|
|
line will appear. For all other configuration properties, you can assume the default value is used.
|
2014-05-21 21:49:12 -04:00
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
## Available Properties
|
2014-05-12 22:44:14 -04:00
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
Most of the properties that control internal settings have reasonable default values. Some
|
|
|
|
of the most common options to set are:
|
2012-09-26 22:17:58 -04:00
|
|
|
|
[SPARK-1566] consolidate programming guide, and general doc updates
This is a fairly large PR to clean up and update the docs for 1.0. The major changes are:
* A unified programming guide for all languages replaces language-specific ones and shows language-specific info in tabs
* New programming guide sections on key-value pairs, unit testing, input formats beyond text, migrating from 0.9, and passing functions to Spark
* Spark-submit guide moved to a separate page and expanded slightly
* Various cleanups of the menu system, security docs, and others
* Updated look of title bar to differentiate the docs from previous Spark versions
You can find the updated docs at http://people.apache.org/~matei/1.0-docs/_site/ and in particular http://people.apache.org/~matei/1.0-docs/_site/programming-guide.html.
Author: Matei Zaharia <matei@databricks.com>
Closes #896 from mateiz/1.0-docs and squashes the following commits:
03e6853 [Matei Zaharia] Some tweaks to configuration and YARN docs
0779508 [Matei Zaharia] tweak
ef671d4 [Matei Zaharia] Keep frames in JavaDoc links, and other small tweaks
1bf4112 [Matei Zaharia] Review comments
4414f88 [Matei Zaharia] tweaks
d04e979 [Matei Zaharia] Fix some old links to Java guide
a34ed33 [Matei Zaharia] tweak
541bb3b [Matei Zaharia] miscellaneous changes
fcefdec [Matei Zaharia] Moved submitting apps to separate doc
61d72b4 [Matei Zaharia] stuff
181f217 [Matei Zaharia] migration guide, remove old language guides
e11a0da [Matei Zaharia] Add more API functions
6a030a9 [Matei Zaharia] tweaks
8db0ae3 [Matei Zaharia] Added key-value pairs section
318d2c9 [Matei Zaharia] tweaks
1c81477 [Matei Zaharia] New section on basics and function syntax
e38f559 [Matei Zaharia] Actually added programming guide to Git
a33d6fe [Matei Zaharia] First pass at updating programming guide to support all languages, plus other tweaks throughout
3b6a876 [Matei Zaharia] More CSS tweaks
01ec8bf [Matei Zaharia] More CSS tweaks
e6d252e [Matei Zaharia] Change color of doc title bar to differentiate from 0.9.0
2014-05-30 03:34:33 -04:00
|
|
|
#### Application Properties
|
2012-09-26 22:17:58 -04:00
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.app.name</code></td>
|
|
|
|
<td>(none)</td>
|
|
|
|
<td>
|
|
|
|
The name of your application. This will appear in the UI and in log data.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2013-06-30 18:38:58 -04:00
|
|
|
<tr>
|
2015-01-16 12:16:56 -05:00
|
|
|
<td><code>spark.driver.cores</code></td>
|
|
|
|
<td>1</td>
|
2013-06-30 18:38:58 -04:00
|
|
|
<td>
|
2015-01-16 12:16:56 -05:00
|
|
|
Number of cores to use for the driver process, only in cluster mode.
|
2013-06-30 18:38:58 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-11-02 02:03:51 -05:00
|
|
|
<td><code>spark.driver.maxResultSize</code></td>
|
|
|
|
<td>1g</td>
|
|
|
|
<td>
|
|
|
|
Limit of total size of serialized results of all partitions for each Spark action (e.g. collect).
|
|
|
|
Should be at least 1M, or 0 for unlimited. Jobs will be aborted if the total size
|
2014-12-11 09:21:23 -05:00
|
|
|
is above this limit.
|
2014-11-02 02:03:51 -05:00
|
|
|
Having a high limit may cause out-of-memory errors in driver (depends on spark.driver.memory
|
|
|
|
and memory overhead of objects in JVM). Setting a proper limit can protect the driver from
|
|
|
|
out-of-memory errors.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2012-09-26 22:17:58 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.driver.memory</code></td>
|
|
|
|
<td>512m</td>
|
2012-09-26 22:17:58 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Amount of memory to use for the driver process, i.e. where SparkContext is initialized.
|
|
|
|
(e.g. <code>512m</code>, <code>2g</code>).
|
|
|
|
|
|
|
|
<br /><em>Note:</em> In client mode, this config must not be set through the <code>SparkConf</code>
|
|
|
|
directly in your application, because the driver JVM has already started at that point.
|
|
|
|
Instead, please set this through the <code>--driver-memory</code> command line option
|
|
|
|
or in your default properties file.
|
2012-09-26 22:17:58 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-10-22 00:53:09 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.executor.memory</code></td>
|
|
|
|
<td>512m</td>
|
2014-10-22 00:53:09 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Amount of memory to use per executor process, in the same format as JVM memory strings
|
|
|
|
(e.g. <code>512m</code>, <code>2g</code>).
|
2014-10-22 00:53:09 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2012-09-26 22:17:58 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.extraListeners</code></td>
|
2012-09-26 22:17:58 -04:00
|
|
|
<td>(none)</td>
|
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
A comma-separated list of classes that implement <code>SparkListener</code>; when initializing
|
|
|
|
SparkContext, instances of these classes will be created and registered with Spark's listener
|
|
|
|
bus. If a class has a single-argument constructor that accepts a SparkConf, that constructor
|
|
|
|
will be called; otherwise, a zero-argument constructor will be called. If no valid constructor
|
|
|
|
can be found, the SparkContext creation will fail with an exception.
|
2012-09-26 22:17:58 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2014-05-21 21:49:12 -04:00
|
|
|
<td><code>spark.local.dir</code></td>
|
2012-09-26 22:17:58 -04:00
|
|
|
<td>/tmp</td>
|
|
|
|
<td>
|
2014-05-25 20:15:47 -04:00
|
|
|
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.
|
2014-04-21 13:26:33 -04:00
|
|
|
|
|
|
|
NOTE: In Spark 1.0 and later this will be overriden by SPARK_LOCAL_DIRS (Standalone, Mesos) or
|
2014-05-28 18:49:54 -04:00
|
|
|
LOCAL_DIRS (YARN) environment variables set by the cluster manager.
|
2012-09-26 22:17:58 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td><code>spark.logConf</code></td>
|
|
|
|
<td>false</td>
|
2012-09-26 22:17:58 -04:00
|
|
|
<td>
|
2014-05-28 18:49:54 -04:00
|
|
|
Logs the effective SparkConf as INFO when a SparkContext is started.
|
2012-09-26 22:17:58 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-04 20:18:03 -05:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.master</code></td>
|
2015-02-04 20:18:03 -05:00
|
|
|
<td>(none)</td>
|
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
The cluster manager to connect to. See the list of
|
|
|
|
<a href="submitting-applications.html#master-urls"> allowed master URL's</a>.
|
2015-02-04 20:18:03 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2012-09-26 22:17:58 -04:00
|
|
|
</table>
|
|
|
|
|
|
|
|
Apart from these, the following properties are also available, and may be useful in some situations:
|
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
#### Runtime Environment
|
2012-09-26 22:17:58 -04:00
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
2015-01-25 18:08:05 -05:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.driver.extraClassPath</code></td>
|
2015-01-25 18:08:05 -05:00
|
|
|
<td>(none)</td>
|
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Extra classpath entries to append to the classpath of the driver.
|
|
|
|
|
2015-02-19 18:50:58 -05:00
|
|
|
<br /><em>Note:</em> In client mode, this config must not be set through the <code>SparkConf</code>
|
|
|
|
directly in your application, because the driver JVM has already started at that point.
|
2015-02-25 19:12:56 -05:00
|
|
|
Instead, please set this through the <code>--driver-class-path</code> command line option or in
|
2015-02-19 18:50:58 -05:00
|
|
|
your default properties file.</td>
|
2015-01-25 18:08:05 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.driver.extraJavaOptions</code></td>
|
2015-01-25 18:08:05 -05:00
|
|
|
<td>(none)</td>
|
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.
|
|
|
|
|
2015-02-19 18:50:58 -05:00
|
|
|
<br /><em>Note:</em> In client mode, this config must not be set through the <code>SparkConf</code>
|
|
|
|
directly in your application, because the driver JVM has already started at that point.
|
2015-02-25 19:12:56 -05:00
|
|
|
Instead, please set this through the <code>--driver-java-options</code> command line option or in
|
2015-02-19 18:50:58 -05:00
|
|
|
your default properties file.</td>
|
2015-01-25 18:08:05 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.driver.extraLibraryPath</code></td>
|
|
|
|
<td>(none)</td>
|
|
|
|
<td>
|
|
|
|
Set a special library path to use when launching the driver JVM.
|
2015-02-19 18:50:58 -05:00
|
|
|
|
|
|
|
<br /><em>Note:</em> In client mode, this config must not be set through the <code>SparkConf</code>
|
|
|
|
directly in your application, because the driver JVM has already started at that point.
|
|
|
|
Instead, please set this through the <code>--driver-library-path</code> command line option or in
|
|
|
|
your default properties file.</td>
|
2015-01-25 18:08:05 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
[SPARK-2996] Implement userClassPathFirst for driver, yarn.
Yarn's config option `spark.yarn.user.classpath.first` does not work the same way as
`spark.files.userClassPathFirst`; Yarn's version is a lot more dangerous, in that it
modifies the system classpath, instead of restricting the changes to the user's class
loader. So this change implements the behavior of the latter for Yarn, and deprecates
the more dangerous choice.
To be able to achieve feature-parity, I also implemented the option for drivers (the existing
option only applies to executors). So now there are two options, each controlling whether
to apply userClassPathFirst to the driver or executors. The old option was deprecated, and
aliased to the new one (`spark.executor.userClassPathFirst`).
The existing "child-first" class loader also had to be fixed. It didn't handle resources, and it
was also doing some things that ended up causing JVM errors depending on how things
were being called.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #3233 from vanzin/SPARK-2996 and squashes the following commits:
9cf9cf1 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1499e2 [Marcelo Vanzin] Remove SPARK_HOME propagation.
fa7df88 [Marcelo Vanzin] Remove 'test.resource' file, create it dynamically.
a8c69f1 [Marcelo Vanzin] Review feedback.
cabf962 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1b8d7e [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3f768e3 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
2ce3c7a [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6d6be [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
70d4044 [Marcelo Vanzin] Fix pyspark/yarn-cluster test.
0fe7777 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6ef19 [Marcelo Vanzin] Move class loaders around and make names more meaninful.
fe970a7 [Marcelo Vanzin] Review feedback.
25d4fed [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3cb6498 [Marcelo Vanzin] Call the right loadClass() method on the parent.
fbb8ab5 [Marcelo Vanzin] Add locking in loadClass() to avoid deadlocks.
2e6c4b7 [Marcelo Vanzin] Mention new setting in documentation.
b6497f9 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a10f379 [Marcelo Vanzin] Some feedback.
3730151 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
f513871 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
44010b6 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
7b57cba [Marcelo Vanzin] Remove now outdated message.
5304d64 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
35949c8 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
54e1a98 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
d1273b2 [Marcelo Vanzin] Add test file to rat exclude.
fa1aafa [Marcelo Vanzin] Remove write check on user jars.
89d8072 [Marcelo Vanzin] Cleanups.
a963ea3 [Marcelo Vanzin] Implement spark.driver.userClassPathFirst for standalone cluster mode.
50afa5f [Marcelo Vanzin] Fix Yarn executor command line.
7d14397 [Marcelo Vanzin] Register user jars in executor up front.
7f8603c [Marcelo Vanzin] Fix yarn-cluster mode without userClassPathFirst.
20373f5 [Marcelo Vanzin] Fix ClientBaseSuite.
55c88fa [Marcelo Vanzin] Run all Yarn integration tests via spark-submit.
0b64d92 [Marcelo Vanzin] Add deprecation warning to yarn option.
4a84d87 [Marcelo Vanzin] Fix the child-first class loader.
d0394b8 [Marcelo Vanzin] Add "deprecated configs" to SparkConf.
46d8cf2 [Marcelo Vanzin] Update doc with new option, change name to "userClassPathFirst".
a314f2d [Marcelo Vanzin] Enable driver class path isolation in SparkSubmit.
91f7e54 [Marcelo Vanzin] [yarn] Enable executor class path isolation.
a853e74 [Marcelo Vanzin] Re-work CoarseGrainedExecutorBackend command line arguments.
89522ef [Marcelo Vanzin] Add class path isolation support for Yarn cluster mode.
2015-02-10 00:17:06 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.driver.userClassPathFirst</code></td>
|
|
|
|
<td>false</td>
|
|
|
|
<td>
|
|
|
|
(Experimental) Whether to give user-added jars precedence over Spark's own jars when loading
|
|
|
|
classes in the the driver. This feature can be used to mitigate conflicts between Spark's
|
|
|
|
dependencies and user dependencies. It is currently an experimental feature.
|
2015-02-19 18:50:58 -05:00
|
|
|
|
|
|
|
This is used in cluster mode only.
|
[SPARK-2996] Implement userClassPathFirst for driver, yarn.
Yarn's config option `spark.yarn.user.classpath.first` does not work the same way as
`spark.files.userClassPathFirst`; Yarn's version is a lot more dangerous, in that it
modifies the system classpath, instead of restricting the changes to the user's class
loader. So this change implements the behavior of the latter for Yarn, and deprecates
the more dangerous choice.
To be able to achieve feature-parity, I also implemented the option for drivers (the existing
option only applies to executors). So now there are two options, each controlling whether
to apply userClassPathFirst to the driver or executors. The old option was deprecated, and
aliased to the new one (`spark.executor.userClassPathFirst`).
The existing "child-first" class loader also had to be fixed. It didn't handle resources, and it
was also doing some things that ended up causing JVM errors depending on how things
were being called.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #3233 from vanzin/SPARK-2996 and squashes the following commits:
9cf9cf1 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1499e2 [Marcelo Vanzin] Remove SPARK_HOME propagation.
fa7df88 [Marcelo Vanzin] Remove 'test.resource' file, create it dynamically.
a8c69f1 [Marcelo Vanzin] Review feedback.
cabf962 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1b8d7e [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3f768e3 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
2ce3c7a [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6d6be [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
70d4044 [Marcelo Vanzin] Fix pyspark/yarn-cluster test.
0fe7777 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6ef19 [Marcelo Vanzin] Move class loaders around and make names more meaninful.
fe970a7 [Marcelo Vanzin] Review feedback.
25d4fed [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3cb6498 [Marcelo Vanzin] Call the right loadClass() method on the parent.
fbb8ab5 [Marcelo Vanzin] Add locking in loadClass() to avoid deadlocks.
2e6c4b7 [Marcelo Vanzin] Mention new setting in documentation.
b6497f9 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a10f379 [Marcelo Vanzin] Some feedback.
3730151 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
f513871 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
44010b6 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
7b57cba [Marcelo Vanzin] Remove now outdated message.
5304d64 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
35949c8 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
54e1a98 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
d1273b2 [Marcelo Vanzin] Add test file to rat exclude.
fa1aafa [Marcelo Vanzin] Remove write check on user jars.
89d8072 [Marcelo Vanzin] Cleanups.
a963ea3 [Marcelo Vanzin] Implement spark.driver.userClassPathFirst for standalone cluster mode.
50afa5f [Marcelo Vanzin] Fix Yarn executor command line.
7d14397 [Marcelo Vanzin] Register user jars in executor up front.
7f8603c [Marcelo Vanzin] Fix yarn-cluster mode without userClassPathFirst.
20373f5 [Marcelo Vanzin] Fix ClientBaseSuite.
55c88fa [Marcelo Vanzin] Run all Yarn integration tests via spark-submit.
0b64d92 [Marcelo Vanzin] Add deprecation warning to yarn option.
4a84d87 [Marcelo Vanzin] Fix the child-first class loader.
d0394b8 [Marcelo Vanzin] Add "deprecated configs" to SparkConf.
46d8cf2 [Marcelo Vanzin] Update doc with new option, change name to "userClassPathFirst".
a314f2d [Marcelo Vanzin] Enable driver class path isolation in SparkSubmit.
91f7e54 [Marcelo Vanzin] [yarn] Enable executor class path isolation.
a853e74 [Marcelo Vanzin] Re-work CoarseGrainedExecutorBackend command line arguments.
89522ef [Marcelo Vanzin] Add class path isolation support for Yarn cluster mode.
2015-02-10 00:17:06 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.executor.extraClassPath</code></td>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td>(none)</td>
|
2012-09-26 22:17:58 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Extra classpath entries to append to the classpath of executors. This exists primarily for
|
|
|
|
backwards-compatibility with older versions of Spark. Users typically should not need to set
|
|
|
|
this option.
|
2012-09-26 22:17:58 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2012-09-27 21:45:44 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.executor.extraJavaOptions</code></td>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td>(none)</td>
|
2012-09-27 21:45:44 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
A string of extra JVM options to pass to executors. For instance, GC settings or other logging.
|
|
|
|
Note that it is illegal to set Spark properties or heap size settings with this option. Spark
|
|
|
|
properties should be set using a SparkConf object or the spark-defaults.conf file used with the
|
|
|
|
spark-submit script. Heap size settings can be set with spark.executor.memory.
|
2012-10-07 14:30:53 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-01-10 18:09:51 -05:00
|
|
|
<tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td><code>spark.executor.extraLibraryPath</code></td>
|
|
|
|
<td>(none)</td>
|
2014-01-10 18:09:51 -05:00
|
|
|
<td>
|
2014-05-28 18:49:54 -04:00
|
|
|
Set a special library path to use when launching executor JVM's.
|
2014-01-10 18:09:51 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.executor.logs.rolling.maxRetainedFiles</code></td>
|
|
|
|
<td>(none)</td>
|
|
|
|
<td>
|
|
|
|
Sets the number of latest rolling log files that are going to be retained by the system.
|
|
|
|
Older log files will be deleted. Disabled by default.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-04-28 15:18:55 -04:00
|
|
|
<td><code>spark.executor.logs.rolling.maxSize</code></td>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td>(none)</td>
|
|
|
|
<td>
|
|
|
|
Set the max size of the file by which the executor logs will be rolled over.
|
2015-04-28 15:18:55 -04:00
|
|
|
Rolling is disabled by default. See <code>spark.executor.logs.rolling.maxRetainedFiles</code>
|
2015-02-25 19:12:56 -05:00
|
|
|
for automatic cleaning of old logs.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-12-11 09:21:23 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.executor.logs.rolling.strategy</code></td>
|
|
|
|
<td>(none)</td>
|
|
|
|
<td>
|
|
|
|
Set the strategy of rolling of executor logs. By default it is disabled. It can
|
|
|
|
be set to "time" (time-based rolling) or "size" (size-based rolling). For "time",
|
|
|
|
use <code>spark.executor.logs.rolling.time.interval</code> to set the rolling interval.
|
|
|
|
For "size", use <code>spark.executor.logs.rolling.size.maxBytes</code> to set
|
|
|
|
the maximum file size for rolling.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.executor.logs.rolling.time.interval</code></td>
|
|
|
|
<td>daily</td>
|
|
|
|
<td>
|
|
|
|
Set the time interval by which the executor logs will be rolled over.
|
|
|
|
Rolling is disabled by default. Valid values are `daily`, `hourly`, `minutely` or
|
|
|
|
any interval in seconds. See <code>spark.executor.logs.rolling.maxRetainedFiles</code>
|
|
|
|
for automatic cleaning of old logs.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-04-27 20:40:56 -04:00
|
|
|
<tr>
|
[SPARK-2996] Implement userClassPathFirst for driver, yarn.
Yarn's config option `spark.yarn.user.classpath.first` does not work the same way as
`spark.files.userClassPathFirst`; Yarn's version is a lot more dangerous, in that it
modifies the system classpath, instead of restricting the changes to the user's class
loader. So this change implements the behavior of the latter for Yarn, and deprecates
the more dangerous choice.
To be able to achieve feature-parity, I also implemented the option for drivers (the existing
option only applies to executors). So now there are two options, each controlling whether
to apply userClassPathFirst to the driver or executors. The old option was deprecated, and
aliased to the new one (`spark.executor.userClassPathFirst`).
The existing "child-first" class loader also had to be fixed. It didn't handle resources, and it
was also doing some things that ended up causing JVM errors depending on how things
were being called.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #3233 from vanzin/SPARK-2996 and squashes the following commits:
9cf9cf1 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1499e2 [Marcelo Vanzin] Remove SPARK_HOME propagation.
fa7df88 [Marcelo Vanzin] Remove 'test.resource' file, create it dynamically.
a8c69f1 [Marcelo Vanzin] Review feedback.
cabf962 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1b8d7e [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3f768e3 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
2ce3c7a [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6d6be [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
70d4044 [Marcelo Vanzin] Fix pyspark/yarn-cluster test.
0fe7777 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6ef19 [Marcelo Vanzin] Move class loaders around and make names more meaninful.
fe970a7 [Marcelo Vanzin] Review feedback.
25d4fed [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3cb6498 [Marcelo Vanzin] Call the right loadClass() method on the parent.
fbb8ab5 [Marcelo Vanzin] Add locking in loadClass() to avoid deadlocks.
2e6c4b7 [Marcelo Vanzin] Mention new setting in documentation.
b6497f9 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a10f379 [Marcelo Vanzin] Some feedback.
3730151 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
f513871 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
44010b6 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
7b57cba [Marcelo Vanzin] Remove now outdated message.
5304d64 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
35949c8 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
54e1a98 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
d1273b2 [Marcelo Vanzin] Add test file to rat exclude.
fa1aafa [Marcelo Vanzin] Remove write check on user jars.
89d8072 [Marcelo Vanzin] Cleanups.
a963ea3 [Marcelo Vanzin] Implement spark.driver.userClassPathFirst for standalone cluster mode.
50afa5f [Marcelo Vanzin] Fix Yarn executor command line.
7d14397 [Marcelo Vanzin] Register user jars in executor up front.
7f8603c [Marcelo Vanzin] Fix yarn-cluster mode without userClassPathFirst.
20373f5 [Marcelo Vanzin] Fix ClientBaseSuite.
55c88fa [Marcelo Vanzin] Run all Yarn integration tests via spark-submit.
0b64d92 [Marcelo Vanzin] Add deprecation warning to yarn option.
4a84d87 [Marcelo Vanzin] Fix the child-first class loader.
d0394b8 [Marcelo Vanzin] Add "deprecated configs" to SparkConf.
46d8cf2 [Marcelo Vanzin] Update doc with new option, change name to "userClassPathFirst".
a314f2d [Marcelo Vanzin] Enable driver class path isolation in SparkSubmit.
91f7e54 [Marcelo Vanzin] [yarn] Enable executor class path isolation.
a853e74 [Marcelo Vanzin] Re-work CoarseGrainedExecutorBackend command line arguments.
89522ef [Marcelo Vanzin] Add class path isolation support for Yarn cluster mode.
2015-02-10 00:17:06 -05:00
|
|
|
<td><code>spark.executor.userClassPathFirst</code></td>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td>false</td>
|
2014-04-27 20:40:56 -04:00
|
|
|
<td>
|
[SPARK-2996] Implement userClassPathFirst for driver, yarn.
Yarn's config option `spark.yarn.user.classpath.first` does not work the same way as
`spark.files.userClassPathFirst`; Yarn's version is a lot more dangerous, in that it
modifies the system classpath, instead of restricting the changes to the user's class
loader. So this change implements the behavior of the latter for Yarn, and deprecates
the more dangerous choice.
To be able to achieve feature-parity, I also implemented the option for drivers (the existing
option only applies to executors). So now there are two options, each controlling whether
to apply userClassPathFirst to the driver or executors. The old option was deprecated, and
aliased to the new one (`spark.executor.userClassPathFirst`).
The existing "child-first" class loader also had to be fixed. It didn't handle resources, and it
was also doing some things that ended up causing JVM errors depending on how things
were being called.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #3233 from vanzin/SPARK-2996 and squashes the following commits:
9cf9cf1 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1499e2 [Marcelo Vanzin] Remove SPARK_HOME propagation.
fa7df88 [Marcelo Vanzin] Remove 'test.resource' file, create it dynamically.
a8c69f1 [Marcelo Vanzin] Review feedback.
cabf962 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1b8d7e [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3f768e3 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
2ce3c7a [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6d6be [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
70d4044 [Marcelo Vanzin] Fix pyspark/yarn-cluster test.
0fe7777 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6ef19 [Marcelo Vanzin] Move class loaders around and make names more meaninful.
fe970a7 [Marcelo Vanzin] Review feedback.
25d4fed [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3cb6498 [Marcelo Vanzin] Call the right loadClass() method on the parent.
fbb8ab5 [Marcelo Vanzin] Add locking in loadClass() to avoid deadlocks.
2e6c4b7 [Marcelo Vanzin] Mention new setting in documentation.
b6497f9 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a10f379 [Marcelo Vanzin] Some feedback.
3730151 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
f513871 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
44010b6 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
7b57cba [Marcelo Vanzin] Remove now outdated message.
5304d64 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
35949c8 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
54e1a98 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
d1273b2 [Marcelo Vanzin] Add test file to rat exclude.
fa1aafa [Marcelo Vanzin] Remove write check on user jars.
89d8072 [Marcelo Vanzin] Cleanups.
a963ea3 [Marcelo Vanzin] Implement spark.driver.userClassPathFirst for standalone cluster mode.
50afa5f [Marcelo Vanzin] Fix Yarn executor command line.
7d14397 [Marcelo Vanzin] Register user jars in executor up front.
7f8603c [Marcelo Vanzin] Fix yarn-cluster mode without userClassPathFirst.
20373f5 [Marcelo Vanzin] Fix ClientBaseSuite.
55c88fa [Marcelo Vanzin] Run all Yarn integration tests via spark-submit.
0b64d92 [Marcelo Vanzin] Add deprecation warning to yarn option.
4a84d87 [Marcelo Vanzin] Fix the child-first class loader.
d0394b8 [Marcelo Vanzin] Add "deprecated configs" to SparkConf.
46d8cf2 [Marcelo Vanzin] Update doc with new option, change name to "userClassPathFirst".
a314f2d [Marcelo Vanzin] Enable driver class path isolation in SparkSubmit.
91f7e54 [Marcelo Vanzin] [yarn] Enable executor class path isolation.
a853e74 [Marcelo Vanzin] Re-work CoarseGrainedExecutorBackend command line arguments.
89522ef [Marcelo Vanzin] Add class path isolation support for Yarn cluster mode.
2015-02-10 00:17:06 -05:00
|
|
|
(Experimental) Same functionality as <code>spark.driver.userClassPathFirst</code>, but
|
|
|
|
applied to executor instances.
|
2014-04-27 20:40:56 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-07-25 01:53:47 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.executorEnv.[EnvironmentVariableName]</code></td>
|
|
|
|
<td>(none)</td>
|
2014-07-25 01:53:47 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Add the environment variable specified by <code>EnvironmentVariableName</code> to the Executor
|
|
|
|
process. The user can specify multiple of these to set multiple environment variables.
|
2014-07-25 01:53:47 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
[SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:
```
============================================================
Profile of RDD<id=3>
============================================================
5146507 function calls (5146487 primitive calls) in 71.094 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge)
20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__)
20 0.017 0.001 0.017 0.001 {cPickle.dumps}
1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>)
20 0.001 0.000 0.001 0.000 {reduce}
21 0.001 0.000 0.001 0.000 {cPickle.loads}
20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames)
41 0.001 0.000 0.001 0.000 serializers.py:461(read_int)
40 0.001 0.000 0.002 0.000 serializers.py:179(_batched)
62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects}
20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>)
20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream)
40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func)
41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream)
40 0.000 0.000 71.072 1.777 rdd.py:304(func)
20 0.000 0.000 71.094 3.555 worker.py:82(process)
```
Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as
```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
284 function calls (276 primitive calls) in 0.001 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream)
4 0.000 0.000 0.000 0.000 {reduce}
12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func)
4 0.000 0.000 0.000 0.000 {cPickle.loads}
4 0.000 0.000 0.000 0.000 {cPickle.dumps}
104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>)
8 0.000 0.000 0.000 0.000 serializers.py:461(read_int)
12 0.000 0.000 0.000 0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".
Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"
This is bugfix of #2351 cc JoshRosen
Author: Davies Liu <davies.liu@gmail.com>
Closes #2556 from davies/profiler and squashes the following commits:
e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
858e74c [Davies Liu] compatitable with python 2.6
7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-30 21:24:57 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.python.profile</code></td>
|
|
|
|
<td>false</td>
|
|
|
|
<td>
|
|
|
|
Enable profiling in Python worker, the profile result will show up by `sc.show_profiles()`,
|
|
|
|
or it will be displayed before the driver exiting. It also can be dumped into disk by
|
|
|
|
`sc.dump_profiles(path)`. If some of the profile results had been displayed maually,
|
|
|
|
they will not be displayed automatically before driver exiting.
|
2015-01-28 16:48:06 -05:00
|
|
|
|
|
|
|
By default the `pyspark.profiler.BasicProfiler` will be used, but this can be overridden by
|
|
|
|
passing a profiler class in as a parameter to the `SparkContext` constructor.
|
[SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:
```
============================================================
Profile of RDD<id=3>
============================================================
5146507 function calls (5146487 primitive calls) in 71.094 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge)
20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__)
20 0.017 0.001 0.017 0.001 {cPickle.dumps}
1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>)
20 0.001 0.000 0.001 0.000 {reduce}
21 0.001 0.000 0.001 0.000 {cPickle.loads}
20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames)
41 0.001 0.000 0.001 0.000 serializers.py:461(read_int)
40 0.001 0.000 0.002 0.000 serializers.py:179(_batched)
62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects}
20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>)
20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream)
40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func)
41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream)
40 0.000 0.000 71.072 1.777 rdd.py:304(func)
20 0.000 0.000 71.094 3.555 worker.py:82(process)
```
Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as
```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
284 function calls (276 primitive calls) in 0.001 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream)
4 0.000 0.000 0.000 0.000 {reduce}
12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func)
4 0.000 0.000 0.000 0.000 {cPickle.loads}
4 0.000 0.000 0.000 0.000 {cPickle.dumps}
104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>)
8 0.000 0.000 0.000 0.000 serializers.py:461(read_int)
12 0.000 0.000 0.000 0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".
Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"
This is bugfix of #2351 cc JoshRosen
Author: Davies Liu <davies.liu@gmail.com>
Closes #2556 from davies/profiler and squashes the following commits:
e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
858e74c [Davies Liu] compatitable with python 2.6
7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-30 21:24:57 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.python.profile.dump</code></td>
|
|
|
|
<td>(none)</td>
|
|
|
|
<td>
|
2014-12-11 09:21:23 -05:00
|
|
|
The directory which is used to dump the profile result before driver exiting.
|
[SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:
```
============================================================
Profile of RDD<id=3>
============================================================
5146507 function calls (5146487 primitive calls) in 71.094 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge)
20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__)
20 0.017 0.001 0.017 0.001 {cPickle.dumps}
1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>)
20 0.001 0.000 0.001 0.000 {reduce}
21 0.001 0.000 0.001 0.000 {cPickle.loads}
20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames)
41 0.001 0.000 0.001 0.000 serializers.py:461(read_int)
40 0.001 0.000 0.002 0.000 serializers.py:179(_batched)
62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects}
20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>)
20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream)
40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func)
41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream)
40 0.000 0.000 71.072 1.777 rdd.py:304(func)
20 0.000 0.000 71.094 3.555 worker.py:82(process)
```
Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as
```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
284 function calls (276 primitive calls) in 0.001 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream)
4 0.000 0.000 0.000 0.000 {reduce}
12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func)
4 0.000 0.000 0.000 0.000 {cPickle.loads}
4 0.000 0.000 0.000 0.000 {cPickle.dumps}
104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>)
8 0.000 0.000 0.000 0.000 serializers.py:461(read_int)
12 0.000 0.000 0.000 0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".
Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"
This is bugfix of #2351 cc JoshRosen
Author: Davies Liu <davies.liu@gmail.com>
Closes #2556 from davies/profiler and squashes the following commits:
e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
858e74c [Davies Liu] compatitable with python 2.6
7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-30 21:24:57 -04:00
|
|
|
The results will be dumped as separated file for each RDD. They can be loaded
|
|
|
|
by ptats.Stats(). If this is specified, the profile result will not be displayed
|
|
|
|
automatically.
|
2014-12-11 09:21:23 -05:00
|
|
|
</td>
|
[SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:
```
============================================================
Profile of RDD<id=3>
============================================================
5146507 function calls (5146487 primitive calls) in 71.094 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge)
20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__)
20 0.017 0.001 0.017 0.001 {cPickle.dumps}
1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>)
20 0.001 0.000 0.001 0.000 {reduce}
21 0.001 0.000 0.001 0.000 {cPickle.loads}
20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames)
41 0.001 0.000 0.001 0.000 serializers.py:461(read_int)
40 0.001 0.000 0.002 0.000 serializers.py:179(_batched)
62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects}
20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>)
20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream)
40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func)
41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream)
40 0.000 0.000 71.072 1.777 rdd.py:304(func)
20 0.000 0.000 71.094 3.555 worker.py:82(process)
```
Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as
```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
284 function calls (276 primitive calls) in 0.001 seconds
Ordered by: internal time, cumulative time
ncalls tottime percall cumtime percall filename:lineno(function)
4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream)
4 0.000 0.000 0.000 0.000 {reduce}
12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func)
4 0.000 0.000 0.000 0.000 {cPickle.loads}
4 0.000 0.000 0.000 0.000 {cPickle.dumps}
104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>)
8 0.000 0.000 0.000 0.000 serializers.py:461(read_int)
12 0.000 0.000 0.000 0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".
Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"
This is bugfix of #2351 cc JoshRosen
Author: Davies Liu <davies.liu@gmail.com>
Closes #2556 from davies/profiler and squashes the following commits:
e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
858e74c [Davies Liu] compatitable with python 2.6
7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-30 21:24:57 -04:00
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.python.worker.memory</code></td>
|
|
|
|
<td>512m</td>
|
|
|
|
<td>
|
|
|
|
Amount of memory to use per python worker process during aggregation, in the same
|
|
|
|
format as JVM memory strings (e.g. <code>512m</code>, <code>2g</code>). If the memory
|
|
|
|
used during aggregation goes above this amount, it will spill the data into disks.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-09-13 19:22:04 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.python.worker.reuse</code></td>
|
|
|
|
<td>true</td>
|
|
|
|
<td>
|
|
|
|
Reuse Python worker or not. If yes, it will use a fixed number of Python workers,
|
|
|
|
does not need to fork() a Python process for every tasks. It will be very useful
|
|
|
|
if there is large broadcast, then the broadcast will not be needed to transfered
|
|
|
|
from JVM to Python worker for every task.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
</table>
|
|
|
|
|
|
|
|
#### Shuffle Behavior
|
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
2014-08-05 16:57:32 -04:00
|
|
|
<tr>
|
2015-04-28 15:18:55 -04:00
|
|
|
<td><code>spark.reducer.maxSizeInFlight</code></td>
|
|
|
|
<td>48m</td>
|
2014-08-05 16:57:32 -04:00
|
|
|
<td>
|
2015-04-28 15:18:55 -04:00
|
|
|
Maximum size of map outputs to fetch simultaneously from each reduce task. Since
|
2015-02-25 19:12:56 -05:00
|
|
|
each output requires us to create a buffer to receive it, this represents a fixed memory
|
|
|
|
overhead per reduce task, so keep it small unless you have a large amount of memory.
|
2014-08-05 16:57:32 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-08-28 14:05:44 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.shuffle.blockTransferService</code></td>
|
|
|
|
<td>netty</td>
|
2014-08-28 14:05:44 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Implementation to use for transferring shuffle and cached blocks between executors. There
|
|
|
|
are two implementations available: <code>netty</code> and <code>nio</code>. Netty-based
|
|
|
|
block transfer is intended to be simpler but equally efficient and is the default option
|
|
|
|
starting in 1.2.
|
2014-08-28 14:05:44 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-10-03 15:58:04 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.shuffle.compress</code></td>
|
|
|
|
<td>true</td>
|
2014-10-03 15:58:04 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Whether to compress map output files. Generally a good idea. Compression will use
|
|
|
|
<code>spark.io.compression.codec</code>.
|
2014-10-03 15:58:04 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-04-04 23:36:24 -04:00
|
|
|
<tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td><code>spark.shuffle.consolidateFiles</code></td>
|
|
|
|
<td>false</td>
|
2014-04-04 23:36:24 -04:00
|
|
|
<td>
|
2014-05-28 18:49:54 -04:00
|
|
|
If set to "true", consolidates intermediate files created during a shuffle. Creating fewer
|
|
|
|
files can improve filesystem performance for shuffles with large numbers of reduce tasks. It
|
|
|
|
is recommended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option
|
|
|
|
might degrade performance on machines with many (>8) cores due to filesystem limitations.
|
2014-04-04 23:36:24 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-04-28 15:18:55 -04:00
|
|
|
<td><code>spark.shuffle.file.buffer</code></td>
|
|
|
|
<td>32k</td>
|
2014-04-04 23:36:24 -04:00
|
|
|
<td>
|
2015-04-28 15:18:55 -04:00
|
|
|
Size of the in-memory buffer for each shuffle file output stream. These buffers
|
2015-02-25 19:12:56 -05:00
|
|
|
reduce the number of disk seeks and system calls made in creating intermediate shuffle files.
|
2014-04-04 23:36:24 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2013-09-06 00:29:37 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.shuffle.io.maxRetries</code></td>
|
|
|
|
<td>3</td>
|
2013-09-06 00:29:37 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
(Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is
|
|
|
|
set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC
|
|
|
|
pauses or transient network connectivity issues.
|
2013-09-06 00:29:37 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2013-02-27 01:52:38 -05:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.shuffle.io.numConnectionsPerPeer</code></td>
|
|
|
|
<td>1</td>
|
2013-02-27 01:52:38 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
(Netty only) Connections between hosts are reused in order to reduce connection buildup for
|
|
|
|
large clusters. For clusters with many hard disks and few hosts, this may result in insufficient
|
|
|
|
concurrency to saturate all disks, and so users may consider increasing this value.
|
2013-06-26 11:54:57 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.shuffle.io.preferDirectBufs</code></td>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td>true</td>
|
2013-06-26 11:54:57 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
(Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache
|
|
|
|
block transfer. For environments where off-heap memory is tightly limited, users may wish to
|
|
|
|
turn this off to force all allocations from Netty to be on-heap.
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.shuffle.io.retryWait</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>5s</td>
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
(Netty only) How long to wait between retries of fetches. The maximum delay caused by retrying
|
|
|
|
is 15 seconds by default, calculated as <code>maxRetries * retryWait</code>.
|
2014-05-28 18:49:54 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-08-07 21:04:49 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.shuffle.manager</code></td>
|
2014-09-07 23:42:07 -04:00
|
|
|
<td>sort</td>
|
2014-08-07 21:04:49 -04:00
|
|
|
<td>
|
2014-09-07 23:42:07 -04:00
|
|
|
Implementation to use for shuffling data. There are two implementations available:
|
|
|
|
<code>sort</code> and <code>hash</code>. Sort-based shuffle is more memory-efficient and is
|
|
|
|
the default option starting in 1.2.
|
2014-08-07 21:04:49 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.shuffle.memoryFraction</code></td>
|
|
|
|
<td>0.2</td>
|
|
|
|
<td>
|
|
|
|
Fraction of Java heap to use for aggregation and cogroups during shuffles, if
|
|
|
|
<code>spark.shuffle.spill</code> is true. At any given time, the collective size of
|
|
|
|
all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will
|
|
|
|
begin to spill to disk. If spills are often, consider increasing this value at the expense of
|
|
|
|
<code>spark.storage.memoryFraction</code>.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-08-07 21:04:49 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.shuffle.sort.bypassMergeThreshold</code></td>
|
|
|
|
<td>200</td>
|
|
|
|
<td>
|
|
|
|
(Advanced) In the sort-based shuffle manager, avoid merge-sorting data if there is no
|
|
|
|
map-side aggregation and there are at most this many reduce partitions.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-11-01 16:15:24 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.shuffle.spill</code></td>
|
|
|
|
<td>true</td>
|
2014-11-01 16:15:24 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
If set to "true", limits the amount of memory used during reduces by spilling data out to disk.
|
|
|
|
This spilling threshold is specified by <code>spark.shuffle.memoryFraction</code>.
|
2014-11-01 16:15:24 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.shuffle.spill.compress</code></td>
|
|
|
|
<td>true</td>
|
|
|
|
<td>
|
|
|
|
Whether to compress data spilled during shuffles. Compression will use
|
|
|
|
<code>spark.io.compression.codec</code>.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
</table>
|
|
|
|
|
|
|
|
#### Spark UI
|
2014-05-28 18:49:54 -04:00
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.eventLog.compress</code></td>
|
|
|
|
<td>false</td>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Whether to compress logged events, if <code>spark.eventLog.enabled</code> is true.
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-04-04 23:36:24 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.eventLog.dir</code></td>
|
|
|
|
<td>file:///tmp/spark-events</td>
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Base directory in which Spark events are logged, if <code>spark.eventLog.enabled</code> is true.
|
|
|
|
Within this base directory, Spark creates a sub-directory for each application, and logs the
|
|
|
|
events specific to the application in this directory. Users may want to set this to
|
|
|
|
a unified location like an HDFS directory so history files can be read by the history server.
|
[SPARK-2321] Stable pull-based progress / status API
This pull request is a first step towards the implementation of a stable, pull-based progress / status API for Spark (see [SPARK-2321](https://issues.apache.org/jira/browse/SPARK-2321)). For now, I'd like to discuss the basic implementation, API names, and overall interface design. Once we arrive at a good design, I'll go back and add additional methods to expose more information via these API.
#### Design goals:
- Pull-based API
- Usable from Java / Scala / Python (eventually, likely with a wrapper)
- Can be extended to expose more information without introducing binary incompatibilities.
- Returns immutable objects.
- Don't leak any implementation details, preserving our freedom to change the implementation.
#### Implementation:
- Add public methods (`getJobInfo`, `getStageInfo`) to SparkContext to allow status / progress information to be retrieved.
- Add public interfaces (`SparkJobInfo`, `SparkStageInfo`) for our API return values. These interfaces consist entirely of Java-style getter methods. The interfaces are currently implemented in Java. I decided to explicitly separate the interface from its implementation (`SparkJobInfoImpl`, `SparkStageInfoImpl`) in order to prevent users from constructing these responses themselves.
-Allow an existing JobProgressListener to be used when constructing a live SparkUI. This allows us to re-use this listeners in the implementation of this status API. There are a few reasons why this listener re-use makes sense:
- The status API and web UI are guaranteed to show consistent information.
- These listeners are already well-tested.
- The same garbage-collection / information retention configurations can apply to both this API and the web UI.
- Extend JobProgressListener to maintain `jobId -> Job` and `stageId -> Stage` mappings.
The progress API methods are implemented in a separate trait that's mixed into SparkContext. This helps to avoid SparkContext.scala from becoming larger and more difficult to read.
Author: Josh Rosen <joshrosen@databricks.com>
Author: Josh Rosen <joshrosen@apache.org>
Closes #2696 from JoshRosen/progress-reporting-api and squashes the following commits:
e6aa78d [Josh Rosen] Add tests.
b585c16 [Josh Rosen] Accept SparkListenerBus instead of more specific subclasses.
c96402d [Josh Rosen] Address review comments.
2707f98 [Josh Rosen] Expose current stage attempt id
c28ba76 [Josh Rosen] Update demo code:
646ff1d [Josh Rosen] Document spark.ui.retainedJobs.
7f47d6d [Josh Rosen] Clean up SparkUI constructors, per Andrew's feedback.
b77b3d8 [Josh Rosen] Merge remote-tracking branch 'origin/master' into progress-reporting-api
787444c [Josh Rosen] Move status API methods into trait that can be mixed into SparkContext.
f9a9a00 [Josh Rosen] More review comments:
3dc79af [Josh Rosen] Remove creation of unused listeners in SparkContext.
249ca16 [Josh Rosen] Address several review comments:
da5648e [Josh Rosen] Add example of basic progress reporting in Java.
7319ffd [Josh Rosen] Add getJobIdsForGroup() and num*Tasks() methods.
cc568e5 [Josh Rosen] Add note explaining that interfaces should not be implemented outside of Spark.
6e840d4 [Josh Rosen] Remove getter-style names and "consistent snapshot" semantics:
08cbec9 [Josh Rosen] Begin to sketch the interfaces for a stable, public status API.
ac2d13a [Josh Rosen] Add jobId->stage, stageId->stage mappings in JobProgressListener
24de263 [Josh Rosen] Create UI listeners in SparkContext instead of in Tabs:
2014-10-25 03:06:57 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.eventLog.enabled</code></td>
|
|
|
|
<td>false</td>
|
[SPARK-2321] Stable pull-based progress / status API
This pull request is a first step towards the implementation of a stable, pull-based progress / status API for Spark (see [SPARK-2321](https://issues.apache.org/jira/browse/SPARK-2321)). For now, I'd like to discuss the basic implementation, API names, and overall interface design. Once we arrive at a good design, I'll go back and add additional methods to expose more information via these API.
#### Design goals:
- Pull-based API
- Usable from Java / Scala / Python (eventually, likely with a wrapper)
- Can be extended to expose more information without introducing binary incompatibilities.
- Returns immutable objects.
- Don't leak any implementation details, preserving our freedom to change the implementation.
#### Implementation:
- Add public methods (`getJobInfo`, `getStageInfo`) to SparkContext to allow status / progress information to be retrieved.
- Add public interfaces (`SparkJobInfo`, `SparkStageInfo`) for our API return values. These interfaces consist entirely of Java-style getter methods. The interfaces are currently implemented in Java. I decided to explicitly separate the interface from its implementation (`SparkJobInfoImpl`, `SparkStageInfoImpl`) in order to prevent users from constructing these responses themselves.
-Allow an existing JobProgressListener to be used when constructing a live SparkUI. This allows us to re-use this listeners in the implementation of this status API. There are a few reasons why this listener re-use makes sense:
- The status API and web UI are guaranteed to show consistent information.
- These listeners are already well-tested.
- The same garbage-collection / information retention configurations can apply to both this API and the web UI.
- Extend JobProgressListener to maintain `jobId -> Job` and `stageId -> Stage` mappings.
The progress API methods are implemented in a separate trait that's mixed into SparkContext. This helps to avoid SparkContext.scala from becoming larger and more difficult to read.
Author: Josh Rosen <joshrosen@databricks.com>
Author: Josh Rosen <joshrosen@apache.org>
Closes #2696 from JoshRosen/progress-reporting-api and squashes the following commits:
e6aa78d [Josh Rosen] Add tests.
b585c16 [Josh Rosen] Accept SparkListenerBus instead of more specific subclasses.
c96402d [Josh Rosen] Address review comments.
2707f98 [Josh Rosen] Expose current stage attempt id
c28ba76 [Josh Rosen] Update demo code:
646ff1d [Josh Rosen] Document spark.ui.retainedJobs.
7f47d6d [Josh Rosen] Clean up SparkUI constructors, per Andrew's feedback.
b77b3d8 [Josh Rosen] Merge remote-tracking branch 'origin/master' into progress-reporting-api
787444c [Josh Rosen] Move status API methods into trait that can be mixed into SparkContext.
f9a9a00 [Josh Rosen] More review comments:
3dc79af [Josh Rosen] Remove creation of unused listeners in SparkContext.
249ca16 [Josh Rosen] Address several review comments:
da5648e [Josh Rosen] Add example of basic progress reporting in Java.
7319ffd [Josh Rosen] Add getJobIdsForGroup() and num*Tasks() methods.
cc568e5 [Josh Rosen] Add note explaining that interfaces should not be implemented outside of Spark.
6e840d4 [Josh Rosen] Remove getter-style names and "consistent snapshot" semantics:
08cbec9 [Josh Rosen] Begin to sketch the interfaces for a stable, public status API.
ac2d13a [Josh Rosen] Add jobId->stage, stageId->stage mappings in JobProgressListener
24de263 [Josh Rosen] Create UI listeners in SparkContext instead of in Tabs:
2014-10-25 03:06:57 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Whether to log Spark events, useful for reconstructing the Web UI after the application has
|
|
|
|
finished.
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-04-10 20:10:11 -04:00
|
|
|
<tr>
|
2014-05-21 21:49:12 -04:00
|
|
|
<td><code>spark.ui.killEnabled</code></td>
|
2014-04-10 20:10:11 -04:00
|
|
|
<td>true</td>
|
|
|
|
<td>
|
|
|
|
Allows stages and corresponding jobs to be killed from the web ui.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2012-10-07 14:30:53 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.ui.port</code></td>
|
|
|
|
<td>4040</td>
|
2012-10-07 14:30:53 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Port for your application's dashboard, which shows memory and workload data.
|
2012-09-27 21:45:44 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-01-13 14:30:09 -05:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.ui.retainedJobs</code></td>
|
|
|
|
<td>1000</td>
|
2014-01-13 14:30:09 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
How many jobs the Spark UI and status APIs remember before garbage
|
|
|
|
collecting.
|
2014-01-13 14:30:09 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.ui.retainedStages</code></td>
|
|
|
|
<td>1000</td>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
How many stages the Spark UI and status APIs remember before garbage
|
|
|
|
collecting.
|
2014-05-28 18:49:54 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
</table>
|
|
|
|
|
|
|
|
#### Compression and Serialization
|
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
2012-09-27 21:45:44 -04:00
|
|
|
<tr>
|
2014-05-21 21:49:12 -04:00
|
|
|
<td><code>spark.broadcast.compress</code></td>
|
2012-10-07 14:30:53 -04:00
|
|
|
<td>true</td>
|
2012-09-27 21:45:44 -04:00
|
|
|
<td>
|
2012-10-07 14:30:53 -04:00
|
|
|
Whether to compress broadcast variables before sending them. Generally a good idea.
|
2012-09-27 21:45:44 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2012-09-26 22:17:58 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.closure.serializer</code></td>
|
|
|
|
<td>org.apache.spark.serializer.<br />JavaSerializer</td>
|
2012-09-26 22:17:58 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Serializer class to use for closures. Currently only the Java serializer is supported.
|
2012-09-26 22:17:58 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2013-07-30 20:12:16 -04:00
|
|
|
<tr>
|
2014-05-21 21:49:12 -04:00
|
|
|
<td><code>spark.io.compression.codec</code></td>
|
2014-08-13 01:50:29 -04:00
|
|
|
<td>snappy</td>
|
2013-07-30 20:12:16 -04:00
|
|
|
<td>
|
2014-10-02 16:47:30 -04:00
|
|
|
The codec used to compress internal data such as RDD partitions, broadcast variables and
|
|
|
|
shuffle outputs. By default, Spark provides three codecs: <code>lz4</code>, <code>lzf</code>,
|
2014-12-11 09:21:23 -05:00
|
|
|
and <code>snappy</code>. You can also use fully qualified class names to specify the codec,
|
|
|
|
e.g.
|
|
|
|
<code>org.apache.spark.io.LZ4CompressionCodec</code>,
|
2014-07-15 16:13:33 -04:00
|
|
|
<code>org.apache.spark.io.LZFCompressionCodec</code>,
|
|
|
|
and <code>org.apache.spark.io.SnappyCompressionCodec</code>.
|
2013-07-30 20:12:16 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-07-15 04:46:57 -04:00
|
|
|
<tr>
|
2015-04-28 15:18:55 -04:00
|
|
|
<td><code>spark.io.compression.lz4.blockSize</code></td>
|
|
|
|
<td>32k</td>
|
2014-07-15 04:46:57 -04:00
|
|
|
<td>
|
2015-04-28 15:18:55 -04:00
|
|
|
Block size used in LZ4 compression, in the case when LZ4 compression codec
|
2014-07-15 04:46:57 -04:00
|
|
|
is used. Lowering this block size will also lower shuffle memory usage when LZ4 is used.
|
2014-03-04 13:28:17 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2012-09-26 22:17:58 -04:00
|
|
|
<tr>
|
2015-04-28 15:18:55 -04:00
|
|
|
<td><code>spark.io.compression.snappy.blockSize</code></td>
|
|
|
|
<td>32k</td>
|
2012-09-26 22:17:58 -04:00
|
|
|
<td>
|
2015-04-28 15:18:55 -04:00
|
|
|
Block size used in Snappy compression, in the case when Snappy compression codec
|
2015-02-25 19:12:56 -05:00
|
|
|
is used. Lowering this block size will also lower shuffle memory usage when Snappy is used.
|
2012-09-26 22:17:58 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.kryo.classesToRegister</code></td>
|
|
|
|
<td>(none)</td>
|
2012-09-26 22:17:58 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
If you use Kryo serialization, give a comma-separated list of custom class names to register
|
|
|
|
with Kryo.
|
|
|
|
See the <a href="tuning.html#data-serialization">tuning guide</a> for more details.
|
2012-09-26 22:17:58 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2013-07-14 21:55:54 -04:00
|
|
|
<tr>
|
2014-05-21 21:49:12 -04:00
|
|
|
<td><code>spark.kryo.referenceTracking</code></td>
|
2013-07-14 21:55:54 -04:00
|
|
|
<td>true</td>
|
|
|
|
<td>
|
|
|
|
Whether to track references to the same object when serializing data with Kryo, which is
|
|
|
|
necessary if your object graphs have loops and useful for efficiency if they contain multiple
|
|
|
|
copies of the same object. Can be disabled to improve performance if you know this is not the
|
|
|
|
case.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-07-23 19:30:06 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.kryo.registrationRequired</code></td>
|
|
|
|
<td>false</td>
|
|
|
|
<td>
|
|
|
|
Whether to require registration with Kryo. If set to 'true', Kryo will throw an exception
|
|
|
|
if an unregistered class is serialized. If set to false (the default), Kryo will write
|
|
|
|
unregistered class names along with each object. Writing class names can cause
|
|
|
|
significant performance overhead, so enabling this option can enforce strictly that a
|
|
|
|
user has not omitted classes from registration.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2012-09-26 22:17:58 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.kryo.registrator</code></td>
|
|
|
|
<td>(none)</td>
|
2012-09-26 22:17:58 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
If you use Kryo serialization, set this class to register your custom classes with Kryo. This
|
|
|
|
property is useful if you need to register your classes in a custom way, e.g. to specify a custom
|
|
|
|
field serializer. Otherwise <code>spark.kryo.classesToRegister</code> is simpler. It should be
|
|
|
|
set to a class that extends
|
|
|
|
<a href="api/scala/index.html#org.apache.spark.serializer.KryoRegistrator">
|
|
|
|
<code>KryoRegistrator</code></a>.
|
|
|
|
See the <a href="tuning.html#data-serialization">tuning guide</a> for more details.
|
2014-07-30 03:18:59 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-04-28 15:18:55 -04:00
|
|
|
<td><code>spark.kryoserializer.buffer.max</code></td>
|
|
|
|
<td>64m</td>
|
2014-07-30 03:18:59 -04:00
|
|
|
<td>
|
2015-04-28 15:18:55 -04:00
|
|
|
Maximum allowable size of Kryo serialization buffer. This must be larger than any
|
2014-07-30 03:18:59 -04:00
|
|
|
object you attempt to serialize. Increase this if you get a "buffer limit exceeded" exception
|
|
|
|
inside Kryo.
|
2012-09-26 22:17:58 -04:00
|
|
|
</td>
|
2012-09-27 02:22:15 -04:00
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
2015-04-28 15:18:55 -04:00
|
|
|
<td><code>spark.kryoserializer.buffer</code></td>
|
|
|
|
<td>64k</td>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td>
|
2015-04-28 15:18:55 -04:00
|
|
|
Initial size of Kryo's serialization buffer. Note that there will be one buffer
|
2015-02-25 19:12:56 -05:00
|
|
|
<i>per core</i> on each worker. This buffer will grow up to
|
|
|
|
<code>spark.kryoserializer.buffer.max.mb</code> if needed.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.rdd.compress</code></td>
|
|
|
|
<td>false</td>
|
|
|
|
<td>
|
|
|
|
Whether to compress serialized RDD partitions (e.g. for
|
|
|
|
<code>StorageLevel.MEMORY_ONLY_SER</code>). Can save substantial space at the cost of some
|
|
|
|
extra CPU time.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.serializer</code></td>
|
|
|
|
<td>org.apache.spark.serializer.<br />JavaSerializer</td>
|
|
|
|
<td>
|
|
|
|
Class to use for serializing objects that will be sent over the network or need to be cached
|
|
|
|
in serialized form. The default of Java serialization works with any Serializable Java object
|
|
|
|
but is quite slow, so we recommend <a href="tuning.html">using
|
|
|
|
<code>org.apache.spark.serializer.KryoSerializer</code> and configuring Kryo serialization</a>
|
|
|
|
when speed is necessary. Can be any subclass of
|
|
|
|
<a href="api/scala/index.html#org.apache.spark.serializer.Serializer">
|
|
|
|
<code>org.apache.spark.Serializer</code></a>.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.serializer.objectStreamReset</code></td>
|
|
|
|
<td>100</td>
|
|
|
|
<td>
|
|
|
|
When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches
|
|
|
|
objects to prevent writing redundant data, however that stops garbage collection of those
|
|
|
|
objects. By calling 'reset' you flush that info from the serializer, and allow old
|
|
|
|
objects to be collected. To turn off this periodic reset set it to -1.
|
|
|
|
By default it will reset the serializer every 100 objects.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
</table>
|
|
|
|
|
|
|
|
#### Execution Behavior
|
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.broadcast.blockSize</code></td>
|
2015-04-28 15:18:55 -04:00
|
|
|
<td>4m</td>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td>
|
2015-04-28 15:18:55 -04:00
|
|
|
Size of each piece of a block for <code>TorrentBroadcastFactory</code>.
|
2015-02-25 19:12:56 -05:00
|
|
|
Too large a value decreases parallelism during broadcast (makes it slower); however, if it is
|
|
|
|
too small, <code>BlockManager</code> might take a performance hit.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.broadcast.factory</code></td>
|
|
|
|
<td>org.apache.spark.broadcast.<br />TorrentBroadcastFactory</td>
|
|
|
|
<td>
|
|
|
|
Which broadcast implementation to use.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.cleaner.ttl</code></td>
|
|
|
|
<td>(infinite)</td>
|
|
|
|
<td>
|
|
|
|
Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks
|
|
|
|
generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be
|
|
|
|
forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in
|
|
|
|
case of Spark Streaming applications). Note that any RDD that persists in memory for more than
|
|
|
|
this duration will be cleared as well.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2015-04-14 16:32:06 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.executor.cores</code></td>
|
|
|
|
<td>1 in YARN mode, all the available cores on the worker in standalone mode.</td>
|
|
|
|
<td>
|
|
|
|
The number of cores to use on each executor. For YARN and standalone mode only.
|
|
|
|
|
|
|
|
In standalone mode, setting this parameter allows an application to run multiple executors on
|
|
|
|
the same worker, provided that there are enough cores on that worker. Otherwise, only one
|
|
|
|
executor per application will run on each worker.
|
|
|
|
</td>
|
|
|
|
</tr>
|
SPARK-942: Do not materialize partitions when DISK_ONLY storage level is used
This is a port of a pull request original targeted at incubator-spark: https://github.com/apache/incubator-spark/pull/180
Essentially if a user returns a generative iterator (from a flatMap operation), when trying to persist the data, Spark would first unroll the iterator into an ArrayBuffer, and then try to figure out if it could store the data. In cases where the user provided an iterator that generated more data then available memory, this would case a crash. With this patch, if the user requests a persist with a 'StorageLevel.DISK_ONLY', the iterator will be unrolled as it is inputed into the serializer.
To do this, two changes where made:
1) The type of the 'values' argument in the putValues method of the BlockStore interface was changed from ArrayBuffer to Iterator (and all code interfacing with this method was modified to connect correctly.
2) The JavaSerializer now calls the ObjectOutputStream 'reset' method every 1000 objects. This was done because the ObjectOutputStream caches objects (thus preventing them from being GC'd) to write more compact serialization. If reset is never called, eventually the memory fills up, if it is called too often then the serialization streams become much larger because of redundant class descriptions.
Author: Kyle Ellrott <kellrott@gmail.com>
Closes #50 from kellrott/iterator-to-disk and squashes the following commits:
9ef7cb8 [Kyle Ellrott] Fixing formatting issues.
60e0c57 [Kyle Ellrott] Fixing issues (formatting, variable names, etc.) from review comments
8aa31cd [Kyle Ellrott] Merge ../incubator-spark into iterator-to-disk
33ac390 [Kyle Ellrott] Merge branch 'iterator-to-disk' of github.com:kellrott/incubator-spark into iterator-to-disk
2f684ea [Kyle Ellrott] Refactoring the BlockManager to replace the Either[Either[A,B]] usage. Now using trait 'Values'. Also modified BlockStore.putBytes call to return PutResult, so that it behaves like putValues.
f70d069 [Kyle Ellrott] Adding docs for spark.serializer.objectStreamReset configuration
7ccc74b [Kyle Ellrott] Moving the 'LargeIteratorSuite' to simply test persistance of iterators. It doesn't try to invoke a OOM error any more
16a4cea [Kyle Ellrott] Streamlined the LargeIteratorSuite unit test. It should now run in ~25 seconds. Confirmed that it still crashes an unpatched copy of Spark.
c2fb430 [Kyle Ellrott] Removing more un-needed array-buffer to iterator conversions
627a8b7 [Kyle Ellrott] Wrapping a few long lines
0f28ec7 [Kyle Ellrott] Adding second putValues to BlockStore interface that accepts an ArrayBuffer (rather then an Iterator). This will allow BlockStores to have slightly different behaviors dependent on whether they get an Iterator or ArrayBuffer. In the case of the MemoryStore, it needs to duplicate and cache an Iterator into an ArrayBuffer, but if handed a ArrayBuffer, it can skip the duplication.
656c33e [Kyle Ellrott] Fixing the JavaSerializer to read from the SparkConf rather then the System property.
8644ee8 [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
00c98e0 [Kyle Ellrott] Making the Java ObjectStreamSerializer reset rate configurable by the system variable 'spark.serializer.objectStreamReset', default is not 10000.
40fe1d7 [Kyle Ellrott] Removing rouge space
31fe08e [Kyle Ellrott] Removing un-needed semi-colons
9df0276 [Kyle Ellrott] Added check to make sure that streamed-to-dist RDD actually returns good data in the LargeIteratorSuite
a6424ba [Kyle Ellrott] Wrapping long line
2eeda75 [Kyle Ellrott] Fixing dumb mistake ("||" instead of "&&")
0e6f808 [Kyle Ellrott] Deleting temp output directory when done
95c7f67 [Kyle Ellrott] Simplifying StorageLevel checks
56f71cd [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
44ec35a [Kyle Ellrott] Adding some comments.
5eb2b7e [Kyle Ellrott] Changing the JavaSerializer reset to occur every 1000 objects.
f403826 [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
81d670c [Kyle Ellrott] Adding unit test for straight to disk iterator methods.
d32992f [Kyle Ellrott] Merge remote-tracking branch 'origin/master' into iterator-to-disk
cac1fad [Kyle Ellrott] Fixing MemoryStore, so that it converts incoming iterators to ArrayBuffer objects. This was previously done higher up the stack.
efe1102 [Kyle Ellrott] Changing CacheManager and BlockManager to pass iterators directly to the serializer when a 'DISK_ONLY' persist is called. This is in response to SPARK-942.
2014-03-06 17:51:00 -05:00
|
|
|
<tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td><code>spark.default.parallelism</code></td>
|
SPARK-942: Do not materialize partitions when DISK_ONLY storage level is used
This is a port of a pull request original targeted at incubator-spark: https://github.com/apache/incubator-spark/pull/180
Essentially if a user returns a generative iterator (from a flatMap operation), when trying to persist the data, Spark would first unroll the iterator into an ArrayBuffer, and then try to figure out if it could store the data. In cases where the user provided an iterator that generated more data then available memory, this would case a crash. With this patch, if the user requests a persist with a 'StorageLevel.DISK_ONLY', the iterator will be unrolled as it is inputed into the serializer.
To do this, two changes where made:
1) The type of the 'values' argument in the putValues method of the BlockStore interface was changed from ArrayBuffer to Iterator (and all code interfacing with this method was modified to connect correctly.
2) The JavaSerializer now calls the ObjectOutputStream 'reset' method every 1000 objects. This was done because the ObjectOutputStream caches objects (thus preventing them from being GC'd) to write more compact serialization. If reset is never called, eventually the memory fills up, if it is called too often then the serialization streams become much larger because of redundant class descriptions.
Author: Kyle Ellrott <kellrott@gmail.com>
Closes #50 from kellrott/iterator-to-disk and squashes the following commits:
9ef7cb8 [Kyle Ellrott] Fixing formatting issues.
60e0c57 [Kyle Ellrott] Fixing issues (formatting, variable names, etc.) from review comments
8aa31cd [Kyle Ellrott] Merge ../incubator-spark into iterator-to-disk
33ac390 [Kyle Ellrott] Merge branch 'iterator-to-disk' of github.com:kellrott/incubator-spark into iterator-to-disk
2f684ea [Kyle Ellrott] Refactoring the BlockManager to replace the Either[Either[A,B]] usage. Now using trait 'Values'. Also modified BlockStore.putBytes call to return PutResult, so that it behaves like putValues.
f70d069 [Kyle Ellrott] Adding docs for spark.serializer.objectStreamReset configuration
7ccc74b [Kyle Ellrott] Moving the 'LargeIteratorSuite' to simply test persistance of iterators. It doesn't try to invoke a OOM error any more
16a4cea [Kyle Ellrott] Streamlined the LargeIteratorSuite unit test. It should now run in ~25 seconds. Confirmed that it still crashes an unpatched copy of Spark.
c2fb430 [Kyle Ellrott] Removing more un-needed array-buffer to iterator conversions
627a8b7 [Kyle Ellrott] Wrapping a few long lines
0f28ec7 [Kyle Ellrott] Adding second putValues to BlockStore interface that accepts an ArrayBuffer (rather then an Iterator). This will allow BlockStores to have slightly different behaviors dependent on whether they get an Iterator or ArrayBuffer. In the case of the MemoryStore, it needs to duplicate and cache an Iterator into an ArrayBuffer, but if handed a ArrayBuffer, it can skip the duplication.
656c33e [Kyle Ellrott] Fixing the JavaSerializer to read from the SparkConf rather then the System property.
8644ee8 [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
00c98e0 [Kyle Ellrott] Making the Java ObjectStreamSerializer reset rate configurable by the system variable 'spark.serializer.objectStreamReset', default is not 10000.
40fe1d7 [Kyle Ellrott] Removing rouge space
31fe08e [Kyle Ellrott] Removing un-needed semi-colons
9df0276 [Kyle Ellrott] Added check to make sure that streamed-to-dist RDD actually returns good data in the LargeIteratorSuite
a6424ba [Kyle Ellrott] Wrapping long line
2eeda75 [Kyle Ellrott] Fixing dumb mistake ("||" instead of "&&")
0e6f808 [Kyle Ellrott] Deleting temp output directory when done
95c7f67 [Kyle Ellrott] Simplifying StorageLevel checks
56f71cd [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
44ec35a [Kyle Ellrott] Adding some comments.
5eb2b7e [Kyle Ellrott] Changing the JavaSerializer reset to occur every 1000 objects.
f403826 [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
81d670c [Kyle Ellrott] Adding unit test for straight to disk iterator methods.
d32992f [Kyle Ellrott] Merge remote-tracking branch 'origin/master' into iterator-to-disk
cac1fad [Kyle Ellrott] Fixing MemoryStore, so that it converts incoming iterators to ArrayBuffer objects. This was previously done higher up the stack.
efe1102 [Kyle Ellrott] Changing CacheManager and BlockManager to pass iterators directly to the serializer when a 'DISK_ONLY' persist is called. This is in response to SPARK-942.
2014-03-06 17:51:00 -05:00
|
|
|
<td>
|
2014-11-10 15:40:41 -05:00
|
|
|
For distributed shuffle operations like <code>reduceByKey</code> and <code>join</code>, the
|
|
|
|
largest number of partitions in a parent RDD. For operations like <code>parallelize</code>
|
|
|
|
with no parent RDDs, it depends on the cluster manager:
|
2014-05-28 18:49:54 -04:00
|
|
|
<ul>
|
|
|
|
<li>Local mode: number of cores on the local machine</li>
|
|
|
|
<li>Mesos fine grained mode: 8</li>
|
|
|
|
<li>Others: total number of cores on all executor nodes or 2, whichever is larger</li>
|
|
|
|
</ul>
|
|
|
|
</td>
|
|
|
|
<td>
|
2014-11-10 15:40:41 -05:00
|
|
|
Default number of partitions in RDDs returned by transformations like <code>join</code>,
|
|
|
|
<code>reduceByKey</code>, and <code>parallelize</code> when not set by user.
|
SPARK-942: Do not materialize partitions when DISK_ONLY storage level is used
This is a port of a pull request original targeted at incubator-spark: https://github.com/apache/incubator-spark/pull/180
Essentially if a user returns a generative iterator (from a flatMap operation), when trying to persist the data, Spark would first unroll the iterator into an ArrayBuffer, and then try to figure out if it could store the data. In cases where the user provided an iterator that generated more data then available memory, this would case a crash. With this patch, if the user requests a persist with a 'StorageLevel.DISK_ONLY', the iterator will be unrolled as it is inputed into the serializer.
To do this, two changes where made:
1) The type of the 'values' argument in the putValues method of the BlockStore interface was changed from ArrayBuffer to Iterator (and all code interfacing with this method was modified to connect correctly.
2) The JavaSerializer now calls the ObjectOutputStream 'reset' method every 1000 objects. This was done because the ObjectOutputStream caches objects (thus preventing them from being GC'd) to write more compact serialization. If reset is never called, eventually the memory fills up, if it is called too often then the serialization streams become much larger because of redundant class descriptions.
Author: Kyle Ellrott <kellrott@gmail.com>
Closes #50 from kellrott/iterator-to-disk and squashes the following commits:
9ef7cb8 [Kyle Ellrott] Fixing formatting issues.
60e0c57 [Kyle Ellrott] Fixing issues (formatting, variable names, etc.) from review comments
8aa31cd [Kyle Ellrott] Merge ../incubator-spark into iterator-to-disk
33ac390 [Kyle Ellrott] Merge branch 'iterator-to-disk' of github.com:kellrott/incubator-spark into iterator-to-disk
2f684ea [Kyle Ellrott] Refactoring the BlockManager to replace the Either[Either[A,B]] usage. Now using trait 'Values'. Also modified BlockStore.putBytes call to return PutResult, so that it behaves like putValues.
f70d069 [Kyle Ellrott] Adding docs for spark.serializer.objectStreamReset configuration
7ccc74b [Kyle Ellrott] Moving the 'LargeIteratorSuite' to simply test persistance of iterators. It doesn't try to invoke a OOM error any more
16a4cea [Kyle Ellrott] Streamlined the LargeIteratorSuite unit test. It should now run in ~25 seconds. Confirmed that it still crashes an unpatched copy of Spark.
c2fb430 [Kyle Ellrott] Removing more un-needed array-buffer to iterator conversions
627a8b7 [Kyle Ellrott] Wrapping a few long lines
0f28ec7 [Kyle Ellrott] Adding second putValues to BlockStore interface that accepts an ArrayBuffer (rather then an Iterator). This will allow BlockStores to have slightly different behaviors dependent on whether they get an Iterator or ArrayBuffer. In the case of the MemoryStore, it needs to duplicate and cache an Iterator into an ArrayBuffer, but if handed a ArrayBuffer, it can skip the duplication.
656c33e [Kyle Ellrott] Fixing the JavaSerializer to read from the SparkConf rather then the System property.
8644ee8 [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
00c98e0 [Kyle Ellrott] Making the Java ObjectStreamSerializer reset rate configurable by the system variable 'spark.serializer.objectStreamReset', default is not 10000.
40fe1d7 [Kyle Ellrott] Removing rouge space
31fe08e [Kyle Ellrott] Removing un-needed semi-colons
9df0276 [Kyle Ellrott] Added check to make sure that streamed-to-dist RDD actually returns good data in the LargeIteratorSuite
a6424ba [Kyle Ellrott] Wrapping long line
2eeda75 [Kyle Ellrott] Fixing dumb mistake ("||" instead of "&&")
0e6f808 [Kyle Ellrott] Deleting temp output directory when done
95c7f67 [Kyle Ellrott] Simplifying StorageLevel checks
56f71cd [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
44ec35a [Kyle Ellrott] Adding some comments.
5eb2b7e [Kyle Ellrott] Changing the JavaSerializer reset to occur every 1000 objects.
f403826 [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
81d670c [Kyle Ellrott] Adding unit test for straight to disk iterator methods.
d32992f [Kyle Ellrott] Merge remote-tracking branch 'origin/master' into iterator-to-disk
cac1fad [Kyle Ellrott] Fixing MemoryStore, so that it converts incoming iterators to ArrayBuffer objects. This was previously done higher up the stack.
efe1102 [Kyle Ellrott] Changing CacheManager and BlockManager to pass iterators directly to the serializer when a 'DISK_ONLY' persist is called. This is in response to SPARK-942.
2014-03-06 17:51:00 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2012-09-26 22:17:58 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.executor.heartbeatInterval</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>10s</td>
|
|
|
|
<td>Interval between each executor's heartbeats to the driver. Heartbeats let
|
2015-02-25 19:12:56 -05:00
|
|
|
the driver know that the executor is still alive and update it with metrics for in-progress
|
|
|
|
tasks.</td>
|
2012-09-26 22:17:58 -04:00
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.files.fetchTimeout</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>60s</td>
|
2012-09-26 22:17:58 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Communication timeout to use when fetching files added through SparkContext.addFile() from
|
2015-04-13 19:28:07 -04:00
|
|
|
the driver.
|
2013-08-15 20:22:49 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2015-03-17 12:33:11 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.files.useFetchCache</code></td>
|
|
|
|
<td>true</td>
|
|
|
|
<td>
|
|
|
|
If set to true (default), file fetching will use a local cache that is shared by executors
|
|
|
|
that belong to the same application, which can improve task launching performance when
|
|
|
|
running many executors on the same host. If set to false, these caching optimizations will
|
|
|
|
be disabled and all executors will fetch their own copies of files. This optimization may be
|
|
|
|
disabled in order to use Spark local directories that reside on NFS filesystems (see
|
|
|
|
<a href="https://issues.apache.org/jira/browse/SPARK-6313">SPARK-6313</a> for more details).
|
|
|
|
</td>
|
|
|
|
</tr>
|
2013-08-15 20:22:49 -04:00
|
|
|
<tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td><code>spark.files.overwrite</code></td>
|
|
|
|
<td>false</td>
|
2013-08-15 20:22:49 -04:00
|
|
|
<td>
|
2014-05-28 18:49:54 -04:00
|
|
|
Whether to overwrite files added through SparkContext.addFile() when the target file exists and
|
|
|
|
its contents do not match those of the source.
|
2013-08-15 20:22:49 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.hadoop.cloneConf</code></td>
|
|
|
|
<td>false</td>
|
|
|
|
<td>If set to true, clones a new Hadoop <code>Configuration</code> object for each task. This
|
|
|
|
option should be enabled to work around <code>Configuration</code> thread-safety issues (see
|
|
|
|
<a href="https://issues.apache.org/jira/browse/SPARK-2546">SPARK-2546</a> for more details).
|
|
|
|
This is disabled by default in order to avoid unexpected performance regressions for jobs that
|
|
|
|
are not affected by these issues.</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.hadoop.validateOutputSpecs</code></td>
|
|
|
|
<td>true</td>
|
|
|
|
<td>If set to true, validates the output specification (e.g. checking if the output directory already exists)
|
|
|
|
used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing
|
|
|
|
output directories. We recommend that users do not disable this except if trying to achieve compatibility with
|
|
|
|
previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand.
|
|
|
|
This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since
|
|
|
|
data may need to be rewritten to pre-existing output directories during checkpoint recovery.</td>
|
2014-05-28 18:49:54 -04:00
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.storage.memoryFraction</code></td>
|
|
|
|
<td>0.6</td>
|
|
|
|
<td>
|
|
|
|
Fraction of Java heap to use for Spark's memory cache. This should not be larger than the "old"
|
|
|
|
generation of objects in the JVM, which by default is given 0.6 of the heap, but you can
|
|
|
|
increase it if you configure your own old generation size.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.storage.memoryMapThreshold</code></td>
|
2015-04-28 15:18:55 -04:00
|
|
|
<td>2m</td>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td>
|
2015-04-28 15:18:55 -04:00
|
|
|
Size of a block above which Spark memory maps when reading a block from disk.
|
2015-02-25 19:12:56 -05:00
|
|
|
This prevents Spark from memory mapping very small blocks. In general, memory
|
|
|
|
mapping has high overhead for blocks close to or below the page size of the operating system.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-07-27 19:08:16 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.storage.unrollFraction</code></td>
|
|
|
|
<td>0.2</td>
|
|
|
|
<td>
|
|
|
|
Fraction of <code>spark.storage.memoryFraction</code> to use for unrolling blocks in memory.
|
|
|
|
This is dynamically allocated by dropping existing blocks when there is not enough free
|
|
|
|
storage space to unroll the new block in its entirety.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<tr>
|
2015-05-01 01:24:31 -04:00
|
|
|
<td><code>spark.externalBlockStore.blockManager</code></td>
|
|
|
|
<td>org.apache.spark.storage.TachyonBlockManager</td>
|
|
|
|
<td>
|
|
|
|
Implementation of external block manager (file system) that store RDDs. The file system's URL is set by
|
|
|
|
<code>spark.externalBlockStore.url</code>.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.externalBlockStore.baseDir</code></td>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td>System.getProperty("java.io.tmpdir")</td>
|
2013-08-15 20:22:49 -04:00
|
|
|
<td>
|
2015-05-01 01:24:31 -04:00
|
|
|
Directories of the external block store that store RDDs. The file system's URL is set by
|
|
|
|
<code>spark.externalBlockStore.url</code> It can also be a comma-separated list of multiple
|
2014-05-28 18:49:54 -04:00
|
|
|
directories on Tachyon file system.
|
2013-08-15 20:22:49 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2013-02-06 17:34:46 -05:00
|
|
|
<tr>
|
2015-05-01 01:24:31 -04:00
|
|
|
<td><code>spark.externalBlockStore.url</code></td>
|
|
|
|
<td>tachyon://localhost:19998 for Tachyon</td>
|
2013-02-06 17:34:46 -05:00
|
|
|
<td>
|
2015-05-01 01:24:31 -04:00
|
|
|
The URL of the underlying external blocker file system in the external block store.
|
2013-02-06 17:34:46 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
</table>
|
|
|
|
|
|
|
|
#### Networking
|
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
2014-04-06 22:17:33 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.akka.failure-detector.threshold</code></td>
|
|
|
|
<td>300.0</td>
|
2014-08-06 03:07:40 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
This is set to a larger value to disable failure detector that comes inbuilt akka. It can be
|
|
|
|
enabled again, if you plan to use this feature (Not recommended). This maps to akka's
|
|
|
|
`akka.remote.transport-failure-detector.threshold`. Tune this in combination of
|
|
|
|
`spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to.
|
2014-04-06 22:17:33 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2013-01-21 16:10:02 -05:00
|
|
|
<tr>
|
2014-05-21 21:49:12 -04:00
|
|
|
<td><code>spark.akka.frameSize</code></td>
|
2013-01-21 16:10:02 -05:00
|
|
|
<td>10</td>
|
|
|
|
<td>
|
|
|
|
Maximum message size to allow in "control plane" communication (for serialized tasks and task
|
2013-01-22 16:25:41 -05:00
|
|
|
results), in MB. Increase this if your tasks need to send back large results to the driver
|
2013-01-21 16:10:02 -05:00
|
|
|
(e.g. using <code>collect()</code> on a large dataset).
|
|
|
|
</td>
|
|
|
|
</tr>
|
2012-10-07 14:30:53 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.akka.heartbeat.interval</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>1000s</td>
|
2015-01-05 13:32:37 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
This is set to a larger value to disable the transport failure detector that comes built in to
|
|
|
|
Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger
|
2015-04-13 19:28:07 -04:00
|
|
|
interval value reduces network overhead and a smaller value ( ~ 1 s) might be more
|
2015-02-25 19:12:56 -05:00
|
|
|
informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses`
|
|
|
|
if you need to. A likely positive use case for using failure detector would be: a sensistive
|
|
|
|
failure detector can help evict rogue executors quickly. However this is usually not the case
|
|
|
|
as GC pauses and network lags are expected in a real Spark cluster. Apart from that enabling
|
|
|
|
this leads to a lot of exchanges of heart beats between nodes leading to flooding the network
|
|
|
|
with those.
|
2015-01-05 13:32:37 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2013-11-26 10:14:47 -05:00
|
|
|
<tr>
|
2014-05-21 21:49:12 -04:00
|
|
|
<td><code>spark.akka.heartbeat.pauses</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>6000s</td>
|
2013-11-26 10:14:47 -05:00
|
|
|
<td>
|
2015-02-23 06:29:25 -05:00
|
|
|
This is set to a larger value to disable the transport failure detector that comes built in to Akka.
|
|
|
|
It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart
|
2015-04-13 19:28:07 -04:00
|
|
|
beat pause for Akka. This can be used to control sensitivity to GC pauses. Tune
|
2015-02-23 06:29:25 -05:00
|
|
|
this along with `spark.akka.heartbeat.interval` if you need to.
|
2013-11-26 10:14:47 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.akka.threads</code></td>
|
|
|
|
<td>4</td>
|
2014-12-22 16:09:22 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Number of actor threads to use for communication. Can be useful to increase on large clusters
|
|
|
|
when the driver has a lot of CPU cores.
|
2014-12-22 16:09:22 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.akka.timeout</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>100s</td>
|
2014-12-22 16:09:22 -05:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
Communication timeout between Spark nodes.
|
2014-12-22 16:09:22 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.blockManager.port</code></td>
|
|
|
|
<td>(random)</td>
|
2014-12-22 16:09:22 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Port for all block managers to listen on. These exist on both the driver and the executors.
|
2014-12-22 16:09:22 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.broadcast.port</code></td>
|
|
|
|
<td>(random)</td>
|
2014-12-22 16:09:22 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Port for the driver's HTTP broadcast server to listen on.
|
|
|
|
This is not relevant for torrent broadcast.
|
2014-12-22 16:09:22 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-01-29 00:51:05 -05:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.driver.host</code></td>
|
|
|
|
<td>(local hostname)</td>
|
2014-01-29 00:51:05 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Hostname or IP address for the driver to listen on.
|
|
|
|
This is used for communicating with the executors and the standalone Master.
|
2013-04-16 13:57:05 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2013-07-28 02:08:00 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.driver.port</code></td>
|
|
|
|
<td>(random)</td>
|
2013-07-28 02:08:00 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Port for the driver to listen on.
|
|
|
|
This is used for communicating with the executors and the standalone Master.
|
2013-07-28 02:08:00 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2013-10-17 16:14:12 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.executor.port</code></td>
|
|
|
|
<td>(random)</td>
|
2013-12-04 21:39:34 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Port for the executor to listen on. This is used for communicating with the driver.
|
2013-12-04 21:39:34 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-01-30 12:33:18 -05:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.fileserver.port</code></td>
|
|
|
|
<td>(random)</td>
|
2014-01-30 12:33:18 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Port for the driver's HTTP file server to listen on.
|
2014-01-30 12:33:18 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-01-10 18:09:51 -05:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.network.timeout</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>120s</td>
|
2014-01-10 18:09:51 -05:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
Default timeout for all network interactions. This config will be used in place of
|
|
|
|
<code>spark.core.connection.ack.wait.timeout</code>, <code>spark.akka.timeout</code>,
|
2015-04-21 21:37:53 -04:00
|
|
|
<code>spark.storage.blockManagerSlaveTimeoutMs</code>,
|
|
|
|
<code>spark.shuffle.io.connectionTimeout</code>, <code>spark.rpc.askTimeout</code> or
|
|
|
|
<code>spark.rpc.lookupTimeout</code> if they are not configured.
|
2014-01-10 18:09:51 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2013-12-04 21:39:34 -05:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.port.maxRetries</code></td>
|
|
|
|
<td>16</td>
|
2013-12-04 21:39:34 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Default maximum number of retries when binding to a port before giving up.
|
2013-12-04 21:39:34 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.replClassServer.port</code></td>
|
|
|
|
<td>(random)</td>
|
2013-12-04 21:39:34 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Port for the driver's HTTP class server to listen on.
|
|
|
|
This is only relevant for the Spark shell.
|
2013-12-04 21:39:34 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2015-04-21 21:37:53 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.rpc.numRetries</code></td>
|
|
|
|
<td>3</td>
|
|
|
|
Number of times to retry before an RPC task gives up.
|
|
|
|
An RPC task will run at most times of this number.
|
|
|
|
<td>
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.rpc.retry.wait</code></td>
|
|
|
|
<td>3s</td>
|
|
|
|
<td>
|
|
|
|
Duration for an RPC ask operation to wait before retrying.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.rpc.askTimeout</code></td>
|
|
|
|
<td>120s</td>
|
|
|
|
<td>
|
|
|
|
Duration for an RPC ask operation to wait before timing out.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.rpc.lookupTimeout</code></td>
|
|
|
|
<td>120s</td>
|
|
|
|
Duration for an RPC remote endpoint lookup operation to wait before timing out.
|
|
|
|
<td>
|
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
</table>
|
|
|
|
|
|
|
|
#### Scheduling
|
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
2013-12-04 21:39:34 -05:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.cores.max</code></td>
|
|
|
|
<td>(not set)</td>
|
2013-12-04 21:39:34 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
When running on a <a href="spark-standalone.html">standalone deploy cluster</a> or a
|
|
|
|
<a href="running-on-mesos.html#mesos-run-modes">Mesos cluster in "coarse-grained"
|
|
|
|
sharing mode</a>, the maximum amount of CPU cores to request for the application from
|
|
|
|
across the cluster (not from each machine). If not set, the default will be
|
|
|
|
<code>spark.deploy.defaultCores</code> on Spark's standalone cluster manager, or
|
|
|
|
infinite (all available cores) on Mesos.
|
2013-12-04 21:39:34 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.localExecution.enabled</code></td>
|
|
|
|
<td>false</td>
|
2013-12-04 21:39:34 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Enables Spark to run certain jobs, such as first() or take() on the driver, without sending
|
|
|
|
tasks to the cluster. This can make certain jobs execute very quickly, but may require
|
|
|
|
shipping a whole partition of data to the driver.
|
2013-12-04 21:39:34 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-01-06 05:21:32 -05:00
|
|
|
<tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td><code>spark.locality.wait</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>3s</td>
|
2014-01-07 14:35:52 -05:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
How long to wait to launch a data-local task before giving up and launching it
|
2014-05-28 18:49:54 -04:00
|
|
|
on a less-local node. The same wait will be used to step through multiple locality levels
|
|
|
|
(process-local, node-local, rack-local and then any). It is also possible to customize the
|
|
|
|
waiting time for each level by setting <code>spark.locality.wait.node</code>, etc.
|
|
|
|
You should increase this setting if your tasks are long and see poor locality, but the
|
|
|
|
default usually works well.
|
2014-01-07 14:35:52 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.locality.wait.node</code></td>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td>spark.locality.wait</td>
|
2014-01-07 14:35:52 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Customize the locality wait for node locality. For example, you can set this to 0 to skip
|
|
|
|
node locality and search immediately for rack locality (if your cluster has rack information).
|
2014-03-09 14:57:06 -04:00
|
|
|
</td>
|
2014-01-17 20:27:25 -05:00
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.locality.wait.process</code></td>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td>spark.locality.wait</td>
|
2014-01-17 20:27:25 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Customize the locality wait for process locality. This affects tasks that attempt to access
|
|
|
|
cached data in a particular executor process.
|
2014-01-07 14:35:52 -05:00
|
|
|
</td>
|
Add timeout for fetch file
Currently, when fetch a file, the connection's connect timeout
and read timeout is based on the default jvm setting, in this change, I change it to
use spark.worker.timeout. This can be usefull, when the
connection status between worker is not perfect. And prevent
prematurely remove task set.
Author: Jiacheng Guo <guojc03@gmail.com>
Closes #98 from guojc/master and squashes the following commits:
abfe698 [Jiacheng Guo] add space according request
2a37c34 [Jiacheng Guo] Add timeout for fetch file Currently, when fetch a file, the connection's connect timeout and read timeout is based on the default jvm setting, in this change, I change it to use spark.worker.timeout. This can be usefull, when the connection status between worker is not perfect. And prevent prematurely remove task set.
2014-03-09 14:37:44 -04:00
|
|
|
</tr>
|
|
|
|
<tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td><code>spark.locality.wait.rack</code></td>
|
|
|
|
<td>spark.locality.wait</td>
|
Add timeout for fetch file
Currently, when fetch a file, the connection's connect timeout
and read timeout is based on the default jvm setting, in this change, I change it to
use spark.worker.timeout. This can be usefull, when the
connection status between worker is not perfect. And prevent
prematurely remove task set.
Author: Jiacheng Guo <guojc03@gmail.com>
Closes #98 from guojc/master and squashes the following commits:
abfe698 [Jiacheng Guo] add space according request
2a37c34 [Jiacheng Guo] Add timeout for fetch file Currently, when fetch a file, the connection's connect timeout and read timeout is based on the default jvm setting, in this change, I change it to use spark.worker.timeout. This can be usefull, when the connection status between worker is not perfect. And prevent prematurely remove task set.
2014-03-09 14:37:44 -04:00
|
|
|
<td>
|
2014-05-28 18:49:54 -04:00
|
|
|
Customize the locality wait for rack locality.
|
Add timeout for fetch file
Currently, when fetch a file, the connection's connect timeout
and read timeout is based on the default jvm setting, in this change, I change it to
use spark.worker.timeout. This can be usefull, when the
connection status between worker is not perfect. And prevent
prematurely remove task set.
Author: Jiacheng Guo <guojc03@gmail.com>
Closes #98 from guojc/master and squashes the following commits:
abfe698 [Jiacheng Guo] add space according request
2a37c34 [Jiacheng Guo] Add timeout for fetch file Currently, when fetch a file, the connection's connect timeout and read timeout is based on the default jvm setting, in this change, I change it to use spark.worker.timeout. This can be usefull, when the connection status between worker is not perfect. And prevent prematurely remove task set.
2014-03-09 14:37:44 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-04-09 01:29:21 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.scheduler.maxRegisteredResourcesWaitingTime</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>30s</td>
|
2014-04-09 01:29:21 -04:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
Maximum amount of time to wait for resources to register before scheduling begins.
|
2014-04-09 01:29:21 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
2014-08-09 01:52:56 -04:00
|
|
|
<td><code>spark.scheduler.minRegisteredResourcesRatio</code></td>
|
2015-02-26 16:07:07 -05:00
|
|
|
<td>0.8 for YARN mode; 0.0 otherwise</td>
|
2014-07-14 16:32:49 -04:00
|
|
|
<td>
|
2014-08-09 01:52:56 -04:00
|
|
|
The minimum ratio of registered resources (registered resources / total expected resources)
|
|
|
|
(resources are executors in yarn mode, CPU cores in standalone mode)
|
2014-12-08 19:28:36 -05:00
|
|
|
to wait for before scheduling begins. Specified as a double between 0.0 and 1.0.
|
2014-08-09 01:52:56 -04:00
|
|
|
Regardless of whether the minimum ratio of resources has been reached,
|
2014-12-11 09:21:23 -05:00
|
|
|
the maximum amount of time it will wait before scheduling begins is controlled by config
|
2014-10-16 22:12:39 -04:00
|
|
|
<code>spark.scheduler.maxRegisteredResourcesWaitingTime</code>.
|
2014-07-14 16:32:49 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.scheduler.mode</code></td>
|
|
|
|
<td>FIFO</td>
|
2014-07-14 16:32:49 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
The <a href="job-scheduling.html#scheduling-within-an-application">scheduling mode</a> between
|
|
|
|
jobs submitted to the same SparkContext. Can be set to <code>FAIR</code>
|
|
|
|
to use fair sharing instead of queueing jobs one after another. Useful for
|
|
|
|
multi-user services.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.scheduler.revive.interval</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>1s</td>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
The interval length for the scheduler to revive the worker resource offers to run tasks.
|
2014-07-14 16:32:49 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
[SPARK-3029] Disable local execution of Spark jobs by default
Currently, local execution of Spark jobs is only used by take(), and it can be problematic as it can load a significant amount of data onto the driver. The worst case scenarios occur if the RDD is cached (guaranteed to load whole partition), has very large elements, or the partition is just large and we apply a filter with high selectivity or computational overhead.
Additionally, jobs that run locally in this manner do not show up in the web UI, and are thus harder to track or understand what is occurring.
This PR adds a flag to disable local execution, which is turned OFF by default, with the intention of perhaps eventually removing this functionality altogether. Removing it now is a tougher proposition since it is part of the public runJob API. An alternative solution would be to limit the flag to take()/first() to avoid impacting any external users of this API, but such usage (or, at least, reliance upon the feature) is hopefully minimal.
Author: Aaron Davidson <aaron@databricks.com>
Closes #1321 from aarondav/allowlocal and squashes the following commits:
136b253 [Aaron Davidson] Fix DAGSchedulerSuite
5599d55 [Aaron Davidson] [RFC] Disable local execution of Spark jobs by default
2014-08-14 04:37:38 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.speculation</code></td>
|
[SPARK-3029] Disable local execution of Spark jobs by default
Currently, local execution of Spark jobs is only used by take(), and it can be problematic as it can load a significant amount of data onto the driver. The worst case scenarios occur if the RDD is cached (guaranteed to load whole partition), has very large elements, or the partition is just large and we apply a filter with high selectivity or computational overhead.
Additionally, jobs that run locally in this manner do not show up in the web UI, and are thus harder to track or understand what is occurring.
This PR adds a flag to disable local execution, which is turned OFF by default, with the intention of perhaps eventually removing this functionality altogether. Removing it now is a tougher proposition since it is part of the public runJob API. An alternative solution would be to limit the flag to take()/first() to avoid impacting any external users of this API, but such usage (or, at least, reliance upon the feature) is hopefully minimal.
Author: Aaron Davidson <aaron@databricks.com>
Closes #1321 from aarondav/allowlocal and squashes the following commits:
136b253 [Aaron Davidson] Fix DAGSchedulerSuite
5599d55 [Aaron Davidson] [RFC] Disable local execution of Spark jobs by default
2014-08-14 04:37:38 -04:00
|
|
|
<td>false</td>
|
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
If set to "true", performs speculative execution of tasks. This means if one or more tasks are
|
|
|
|
running slowly in a stage, they will be re-launched.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.speculation.interval</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>100ms</td>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
How often Spark will check for tasks to speculate.
|
2015-02-25 19:12:56 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.speculation.multiplier</code></td>
|
|
|
|
<td>1.5</td>
|
|
|
|
<td>
|
|
|
|
How many times slower a task is than the median to be considered for speculation.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.speculation.quantile</code></td>
|
|
|
|
<td>0.75</td>
|
|
|
|
<td>
|
|
|
|
Percentage of tasks which must be complete before speculation is enabled for a particular stage.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.task.cpus</code></td>
|
|
|
|
<td>1</td>
|
|
|
|
<td>
|
|
|
|
Number of cores to allocate for each task.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.task.maxFailures</code></td>
|
|
|
|
<td>4</td>
|
|
|
|
<td>
|
|
|
|
Number of individual task failures before giving up on the job.
|
|
|
|
Should be greater than or equal to 1. Number of allowed retries = this value - 1.
|
[SPARK-3029] Disable local execution of Spark jobs by default
Currently, local execution of Spark jobs is only used by take(), and it can be problematic as it can load a significant amount of data onto the driver. The worst case scenarios occur if the RDD is cached (guaranteed to load whole partition), has very large elements, or the partition is just large and we apply a filter with high selectivity or computational overhead.
Additionally, jobs that run locally in this manner do not show up in the web UI, and are thus harder to track or understand what is occurring.
This PR adds a flag to disable local execution, which is turned OFF by default, with the intention of perhaps eventually removing this functionality altogether. Removing it now is a tougher proposition since it is part of the public runJob API. An alternative solution would be to limit the flag to take()/first() to avoid impacting any external users of this API, but such usage (or, at least, reliance upon the feature) is hopefully minimal.
Author: Aaron Davidson <aaron@databricks.com>
Closes #1321 from aarondav/allowlocal and squashes the following commits:
136b253 [Aaron Davidson] Fix DAGSchedulerSuite
5599d55 [Aaron Davidson] [RFC] Disable local execution of Spark jobs by default
2014-08-14 04:37:38 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
</table>
|
|
|
|
|
2015-02-25 19:12:56 -05:00
|
|
|
#### Dynamic Allocation
|
2014-12-19 22:36:20 -05:00
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.dynamicAllocation.enabled</code></td>
|
|
|
|
<td>false</td>
|
|
|
|
<td>
|
|
|
|
Whether to use dynamic resource allocation, which scales the number of executors registered
|
|
|
|
with this application up and down based on the workload. Note that this is currently only
|
|
|
|
available on YARN mode. For more detail, see the description
|
|
|
|
<a href="job-scheduling.html#dynamic-resource-allocation">here</a>.
|
|
|
|
<br><br>
|
2015-02-02 15:27:08 -05:00
|
|
|
This requires <code>spark.shuffle.service.enabled</code> to be set.
|
|
|
|
The following configurations are also relevant:
|
2014-12-19 22:36:20 -05:00
|
|
|
<code>spark.dynamicAllocation.minExecutors</code>,
|
|
|
|
<code>spark.dynamicAllocation.maxExecutors</code>, and
|
2015-02-02 15:27:08 -05:00
|
|
|
<code>spark.dynamicAllocation.initialExecutors</code>
|
2014-12-19 22:36:20 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.dynamicAllocation.executorIdleTimeout</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>600s</td>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
If dynamic allocation is enabled and an executor has been idle for more than this duration,
|
|
|
|
the executor will be removed. For more detail, see this
|
2015-02-25 19:12:56 -05:00
|
|
|
<a href="job-scheduling.html#resource-allocation-policy">description</a>.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.dynamicAllocation.initialExecutors</code></td>
|
2014-12-19 22:36:20 -05:00
|
|
|
<td><code>spark.dynamicAllocation.minExecutors</code></td>
|
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Initial number of executors to run if dynamic allocation is enabled.
|
2014-12-19 22:36:20 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.dynamicAllocation.maxExecutors</code></td>
|
2015-02-02 15:27:08 -05:00
|
|
|
<td>Integer.MAX_VALUE</td>
|
|
|
|
<td>
|
|
|
|
Upper bound for the number of executors if dynamic allocation is enabled.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.dynamicAllocation.minExecutors</code></td>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td>0</td>
|
2014-12-19 22:36:20 -05:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Lower bound for the number of executors if dynamic allocation is enabled.
|
2014-12-19 22:36:20 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.dynamicAllocation.schedulerBacklogTimeout</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>5s</td>
|
2014-12-19 22:36:20 -05:00
|
|
|
<td>
|
|
|
|
If dynamic allocation is enabled and there have been pending tasks backlogged for more than
|
2015-04-13 19:28:07 -04:00
|
|
|
this duration, new executors will be requested. For more detail, see this
|
2014-12-19 22:36:20 -05:00
|
|
|
<a href="job-scheduling.html#resource-allocation-policy">description</a>.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.dynamicAllocation.sustainedSchedulerBacklogTimeout</code></td>
|
|
|
|
<td><code>schedulerBacklogTimeout</code></td>
|
|
|
|
<td>
|
|
|
|
Same as <code>spark.dynamicAllocation.schedulerBacklogTimeout</code>, but used only for
|
|
|
|
subsequent executor requests. For more detail, see this
|
|
|
|
<a href="job-scheduling.html#resource-allocation-policy">description</a>.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
</table>
|
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
#### Security
|
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.acls.enable</code></td>
|
|
|
|
<td>false</td>
|
|
|
|
<td>
|
|
|
|
Whether Spark acls should are enabled. If enabled, this checks to see if the user has
|
|
|
|
access permissions to view or modify the job. Note this requires the user to be known,
|
|
|
|
so if the user comes across as null no checks are done. Filters can be used with the UI
|
|
|
|
to authenticate and set the user.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.admin.acls</code></td>
|
|
|
|
<td>Empty</td>
|
|
|
|
<td>
|
|
|
|
Comma separated list of users/administrators that have view and modify access to all Spark jobs.
|
|
|
|
This can be used if you run on a shared cluster and have a set of administrators or devs who
|
|
|
|
help debug when things work.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-04-04 23:36:24 -04:00
|
|
|
<tr>
|
2014-05-21 21:49:12 -04:00
|
|
|
<td><code>spark.authenticate</code></td>
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
<td>false</td>
|
|
|
|
<td>
|
2014-05-28 18:49:54 -04:00
|
|
|
Whether Spark authenticates its internal connections. See
|
|
|
|
<code>spark.authenticate.secret</code> if not running on YARN.
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-04-04 23:36:24 -04:00
|
|
|
<tr>
|
2014-05-21 21:49:12 -04:00
|
|
|
<td><code>spark.authenticate.secret</code></td>
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
<td>None</td>
|
|
|
|
<td>
|
|
|
|
Set the secret key used for Spark to authenticate between components. This needs to be set if
|
2014-05-28 18:49:54 -04:00
|
|
|
not running on YARN and authentication is enabled.
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.core.connection.ack.wait.timeout</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>60s</td>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
How long for the connection to wait for ack to occur before timing
|
2015-02-25 19:12:56 -05:00
|
|
|
out and giving up. To avoid unwilling timeout caused by long pause like GC,
|
|
|
|
you can set larger value.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2014-04-04 23:36:24 -04:00
|
|
|
<tr>
|
2014-05-21 21:49:12 -04:00
|
|
|
<td><code>spark.core.connection.auth.wait.timeout</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>30s</td>
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
How long for the connection to wait for authentication to occur before timing
|
2014-04-04 23:36:24 -04:00
|
|
|
out and giving up.
|
SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request. was https://github.com/apache/incubator-spark/pull/332.
Author: Thomas Graves <tgraves@apache.org>
Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:
dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 19:27:50 -05:00
|
|
|
</td>
|
2014-01-07 14:35:52 -05:00
|
|
|
</tr>
|
2014-08-16 17:15:58 -04:00
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.modify.acls</code></td>
|
|
|
|
<td>Empty</td>
|
2014-08-16 17:15:58 -04:00
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
Comma separated list of users that have modify access to the Spark job. By default only the
|
|
|
|
user that started the Spark job has access to modify it (kill it for example).
|
2014-08-16 17:15:58 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-03-25 16:05:30 -04:00
|
|
|
<tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td><code>spark.ui.filters</code></td>
|
|
|
|
<td>None</td>
|
2014-03-25 16:05:30 -04:00
|
|
|
<td>
|
2014-05-28 18:49:54 -04:00
|
|
|
Comma separated list of filter class names to apply to the Spark web UI. The filter should be a
|
|
|
|
standard <a href="http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html">
|
|
|
|
javax servlet Filter</a>. Parameters to each filter can also be specified by setting a
|
|
|
|
java system property of: <br />
|
|
|
|
<code>spark.<class name of filter>.params='param1=value1,param2=value2'</code><br />
|
|
|
|
For example: <br />
|
|
|
|
<code>-Dspark.ui.filters=com.test.filter1</code> <br />
|
|
|
|
<code>-Dspark.com.test.filter1.params='param1=foo,param2=testing'</code>
|
2014-03-25 16:05:30 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-04-21 13:26:33 -04:00
|
|
|
<tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td><code>spark.ui.view.acls</code></td>
|
|
|
|
<td>Empty</td>
|
2014-04-21 13:26:33 -04:00
|
|
|
<td>
|
2014-05-28 18:49:54 -04:00
|
|
|
Comma separated list of users that have view access to the Spark web ui. By default only the
|
|
|
|
user that started the Spark job has view access.
|
2014-04-21 13:26:33 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
</table>
|
|
|
|
|
2015-02-02 20:18:54 -05:00
|
|
|
#### Encryption
|
|
|
|
|
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.ssl.enabled</code></td>
|
|
|
|
<td>false</td>
|
|
|
|
<td>
|
|
|
|
<p>Whether to enable SSL connections on all supported protocols.</p>
|
|
|
|
|
|
|
|
<p>All the SSL settings like <code>spark.ssl.xxx</code> where <code>xxx</code> is a
|
|
|
|
particular configuration property, denote the global configuration for all the supported
|
|
|
|
protocols. In order to override the global configuration for the particular protocol,
|
|
|
|
the properties must be overwritten in the protocol-specific namespace.</p>
|
|
|
|
|
|
|
|
<p>Use <code>spark.ssl.YYY.XXX</code> settings to overwrite the global configuration for
|
|
|
|
particular protocol denoted by <code>YYY</code>. Currently <code>YYY</code> can be
|
|
|
|
either <code>akka</code> for Akka based connections or <code>fs</code> for broadcast and
|
|
|
|
file server.</p>
|
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.ssl.enabledAlgorithms</code></td>
|
|
|
|
<td>Empty</td>
|
|
|
|
<td>
|
|
|
|
A comma separated list of ciphers. The specified ciphers must be supported by JVM.
|
|
|
|
The reference list of protocols one can find on
|
|
|
|
<a href="https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https">this</a>
|
|
|
|
page.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.ssl.keyPassword</code></td>
|
|
|
|
<td>None</td>
|
|
|
|
<td>
|
|
|
|
A password to the private key in key-store.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2015-02-02 20:18:54 -05:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.ssl.keyStore</code></td>
|
|
|
|
<td>None</td>
|
|
|
|
<td>
|
|
|
|
A path to a key-store file. The path can be absolute or relative to the directory where
|
|
|
|
the component is started in.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.ssl.keyStorePassword</code></td>
|
|
|
|
<td>None</td>
|
|
|
|
<td>
|
|
|
|
A password to the key-store.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-02-25 19:12:56 -05:00
|
|
|
<td><code>spark.ssl.protocol</code></td>
|
2015-02-02 20:18:54 -05:00
|
|
|
<td>None</td>
|
|
|
|
<td>
|
2015-02-25 19:12:56 -05:00
|
|
|
A protocol name. The protocol must be supported by JVM. The reference list of protocols
|
|
|
|
one can find on <a href="https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https">this</a>
|
|
|
|
page.
|
2015-02-02 20:18:54 -05:00
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.ssl.trustStore</code></td>
|
|
|
|
<td>None</td>
|
|
|
|
<td>
|
|
|
|
A path to a trust-store file. The path can be absolute or relative to the directory
|
|
|
|
where the component is started in.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>spark.ssl.trustStorePassword</code></td>
|
|
|
|
<td>None</td>
|
|
|
|
<td>
|
|
|
|
A password to the trust-store.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
</table>
|
|
|
|
|
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
#### Spark Streaming
|
|
|
|
<table class="table">
|
|
|
|
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
|
2014-04-21 13:26:33 -04:00
|
|
|
<tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<td><code>spark.streaming.blockInterval</code></td>
|
2015-04-13 19:28:07 -04:00
|
|
|
<td>200ms</td>
|
2014-04-21 13:26:33 -04:00
|
|
|
<td>
|
2015-04-13 19:28:07 -04:00
|
|
|
Interval at which data received by Spark Streaming receivers is chunked
|
2014-12-11 09:21:23 -05:00
|
|
|
into blocks of data before storing them in Spark. Minimum recommended - 50 ms. See the
|
|
|
|
<a href="streaming-programming-guide.html#level-of-parallelism-in-data-receiving">performance
|
|
|
|
tuning</a> section in the Spark Streaming programing guide for more details.
|
2014-05-28 18:49:54 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-07-10 19:01:08 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.streaming.receiver.maxRate</code></td>
|
2015-03-11 21:48:21 -04:00
|
|
|
<td>not set</td>
|
2014-07-10 19:01:08 -04:00
|
|
|
<td>
|
2015-03-11 21:48:21 -04:00
|
|
|
Maximum rate (number of records per second) at which each receiver will receive data.
|
2014-12-11 09:21:23 -05:00
|
|
|
Effectively, each stream will consume at most this number of records per second.
|
2014-07-10 19:01:08 -04:00
|
|
|
Setting this configuration to 0 or a negative number will put no limit on the rate.
|
2014-12-11 09:21:23 -05:00
|
|
|
See the <a href="streaming-programming-guide.html#deploying-applications">deployment guide</a>
|
|
|
|
in the Spark Streaming programing guide for mode details.
|
|
|
|
</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
2015-01-13 13:07:19 -05:00
|
|
|
<td><code>spark.streaming.receiver.writeAheadLog.enable</code></td>
|
2014-12-11 09:21:23 -05:00
|
|
|
<td>false</td>
|
|
|
|
<td>
|
|
|
|
Enable write ahead logs for receivers. All the input data received through receivers
|
|
|
|
will be saved to write ahead logs that will allow it to be recovered after driver failures.
|
|
|
|
See the <a href="streaming-programming-guide.html#deploying-applications">deployment guide</a>
|
|
|
|
in the Spark Streaming programing guide for more details.
|
2014-07-10 19:01:08 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2014-05-28 18:49:54 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.streaming.unpersist</code></td>
|
|
|
|
<td>true</td>
|
|
|
|
<td>
|
|
|
|
Force RDDs generated and persisted by Spark Streaming to be automatically unpersisted from
|
|
|
|
Spark's memory. The raw input data received by Spark Streaming is also automatically cleared.
|
|
|
|
Setting this to false will allow the raw data and persisted RDDs to be accessible outside the
|
|
|
|
streaming application as they will not be cleared automatically. But it comes at the cost of
|
|
|
|
higher memory usage in Spark.
|
2014-04-21 13:26:33 -04:00
|
|
|
</td>
|
|
|
|
</tr>
|
2015-03-11 21:48:21 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.streaming.kafka.maxRatePerPartition</code></td>
|
|
|
|
<td>not set</td>
|
|
|
|
<td>
|
|
|
|
Maximum rate (number of records per second) at which data will be read from each Kafka
|
|
|
|
partition when using the new Kafka direct stream API. See the
|
|
|
|
<a href="streaming-kafka-integration.html">Kafka Integration guide</a>
|
|
|
|
for more details.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2015-05-02 18:41:14 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.streaming.kafka.maxRetries</code></td>
|
|
|
|
<td>1</td>
|
|
|
|
<td>
|
|
|
|
Maximum number of consecutive retries the driver will make in order to find
|
|
|
|
the latest offsets on the leader of each partition (a default value of 1
|
|
|
|
means that the driver will make a maximum of 2 attempts). Only applies to
|
|
|
|
the new Kafka direct stream API.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2015-05-05 16:42:23 -04:00
|
|
|
<tr>
|
|
|
|
<td><code>spark.streaming.ui.retainedBatches</code></td>
|
|
|
|
<td>1000</td>
|
|
|
|
<td>
|
|
|
|
How many batches the Spark Streaming UI and status APIs remember before garbage collecting.
|
|
|
|
</td>
|
|
|
|
</tr>
|
2012-09-26 22:17:58 -04:00
|
|
|
</table>
|
2012-09-03 02:05:40 -04:00
|
|
|
|
[SPARK-1566] consolidate programming guide, and general doc updates
This is a fairly large PR to clean up and update the docs for 1.0. The major changes are:
* A unified programming guide for all languages replaces language-specific ones and shows language-specific info in tabs
* New programming guide sections on key-value pairs, unit testing, input formats beyond text, migrating from 0.9, and passing functions to Spark
* Spark-submit guide moved to a separate page and expanded slightly
* Various cleanups of the menu system, security docs, and others
* Updated look of title bar to differentiate the docs from previous Spark versions
You can find the updated docs at http://people.apache.org/~matei/1.0-docs/_site/ and in particular http://people.apache.org/~matei/1.0-docs/_site/programming-guide.html.
Author: Matei Zaharia <matei@databricks.com>
Closes #896 from mateiz/1.0-docs and squashes the following commits:
03e6853 [Matei Zaharia] Some tweaks to configuration and YARN docs
0779508 [Matei Zaharia] tweak
ef671d4 [Matei Zaharia] Keep frames in JavaDoc links, and other small tweaks
1bf4112 [Matei Zaharia] Review comments
4414f88 [Matei Zaharia] tweaks
d04e979 [Matei Zaharia] Fix some old links to Java guide
a34ed33 [Matei Zaharia] tweak
541bb3b [Matei Zaharia] miscellaneous changes
fcefdec [Matei Zaharia] Moved submitting apps to separate doc
61d72b4 [Matei Zaharia] stuff
181f217 [Matei Zaharia] migration guide, remove old language guides
e11a0da [Matei Zaharia] Add more API functions
6a030a9 [Matei Zaharia] tweaks
8db0ae3 [Matei Zaharia] Added key-value pairs section
318d2c9 [Matei Zaharia] tweaks
1c81477 [Matei Zaharia] New section on basics and function syntax
e38f559 [Matei Zaharia] Actually added programming guide to Git
a33d6fe [Matei Zaharia] First pass at updating programming guide to support all languages, plus other tweaks throughout
3b6a876 [Matei Zaharia] More CSS tweaks
01ec8bf [Matei Zaharia] More CSS tweaks
e6d252e [Matei Zaharia] Change color of doc title bar to differentiate from 0.9.0
2014-05-30 03:34:33 -04:00
|
|
|
#### Cluster Managers
|
2014-07-23 19:30:06 -04:00
|
|
|
Each cluster manager in Spark has additional configuration options. Configurations
|
2014-05-28 18:49:54 -04:00
|
|
|
can be found on the pages for each mode:
|
|
|
|
|
2015-03-14 13:28:13 -04:00
|
|
|
##### [YARN](running-on-yarn.html#configuration)
|
|
|
|
|
|
|
|
##### [Mesos](running-on-mesos.html#configuration)
|
|
|
|
|
|
|
|
##### [Standalone Mode](spark-standalone.html#cluster-launch-scripts)
|
2014-05-28 18:49:54 -04:00
|
|
|
|
2013-08-31 17:21:10 -04:00
|
|
|
# Environment Variables
|
|
|
|
|
2014-05-25 20:15:47 -04:00
|
|
|
Certain Spark settings can be configured through environment variables, which are read from the
|
|
|
|
`conf/spark-env.sh` script in the directory where Spark is installed (or `conf/spark-env.cmd` on
|
|
|
|
Windows). In Standalone and Mesos modes, this file can give machine specific information such as
|
|
|
|
hostnames. It is also sourced when running local Spark applications or submission scripts.
|
2013-08-31 17:21:10 -04:00
|
|
|
|
2014-05-25 20:15:47 -04:00
|
|
|
Note that `conf/spark-env.sh` does not exist by default when Spark is installed. However, you can
|
|
|
|
copy `conf/spark-env.sh.template` to create it. Make sure you make the copy executable.
|
2013-08-31 17:21:10 -04:00
|
|
|
|
|
|
|
The following variables can be set in `spark-env.sh`:
|
|
|
|
|
2014-05-21 21:49:12 -04:00
|
|
|
|
|
|
|
<table class="table">
|
|
|
|
<tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>JAVA_HOME</code></td>
|
|
|
|
<td>Location where Java is installed (if it's not on your default `PATH`).</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>PYSPARK_PYTHON</code></td>
|
|
|
|
<td>Python binary executable to use for PySpark.</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>SPARK_LOCAL_IP</code></td>
|
|
|
|
<td>IP address of the machine to bind to.</td>
|
|
|
|
</tr>
|
|
|
|
<tr>
|
|
|
|
<td><code>SPARK_PUBLIC_DNS</code></td>
|
|
|
|
<td>Hostname your Spark program will advertise to other machines.</td>
|
|
|
|
</tr>
|
|
|
|
</table>
|
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
In addition to the above, there are also options for setting up the Spark
|
|
|
|
[standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores
|
|
|
|
to use on each machine and maximum memory.
|
2013-08-31 17:21:10 -04:00
|
|
|
|
2014-05-28 18:49:54 -04:00
|
|
|
Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, you might
|
|
|
|
compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface.
|
2013-08-31 17:21:10 -04:00
|
|
|
|
2012-09-26 22:17:58 -04:00
|
|
|
# Configuring Logging
|
2012-09-03 02:05:40 -04:00
|
|
|
|
2014-05-25 20:15:47 -04:00
|
|
|
Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a
|
|
|
|
`log4j.properties` file in the `conf` directory. One way to start is to copy the existing
|
|
|
|
`log4j.properties.template` located there.
|
2014-10-03 13:03:15 -04:00
|
|
|
|
|
|
|
# Overriding configuration directory
|
|
|
|
|
|
|
|
To specify a different configuration directory other than the default "SPARK_HOME/conf",
|
|
|
|
you can set SPARK_CONF_DIR. Spark will use the the configuration files (spark-defaults.conf, spark-env.sh, log4j.properties, etc)
|
|
|
|
from this directory.
|
|
|
|
|