2015-05-29 23:35:14 -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")
2013-12-30 22:17:28 -05:00
val sc = new SparkContext(conf)
2012-09-26 22:17:58 -04:00
{% endhighlight %}
2015-09-21 14:46:39 -04:00
Note that we can have more than 1 thread in local mode, and in cases like Spark Streaming, we may
2015-11-25 14:37:42 -05:00
actually require more than 1 thread to prevent any sort of starvation issues.
2014-11-05 18:45:34 -05:00
2015-09-21 14:46:39 -04:00
Properties that specify some time duration should be configured with a unit of time.
2015-04-13 19:28:07 -04:00
The following format is accepted:
2015-09-21 14:46:39 -04:00
2015-04-13 19:28:07 -04:00
25ms (milliseconds)
5s (seconds)
10m or 10min (minutes)
3h (hours)
5d (days)
1y (years)
2015-09-21 14:46:39 -04:00
2015-12-15 21:24:23 -05:00
Properties that specify a byte size should be configured with a unit of size.
2015-04-28 15:18:55 -04:00
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
2017-01-07 14:15:51 -05:00
2014-05-28 18:49:54 -04:00
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 %}
2015-09-20 00:40:21 -04:00
./bin/spark-submit --name "My app" --master local[4] --conf spark.eventLog.enabled=false
2014-12-11 09:21:23 -05:00
--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
2015-07-22 18:28:09 -04:00
spark.executor.memory 4g
2014-05-12 22:44:14 -04:00
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
2017-01-07 14:15:51 -05: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 >
2016-09-17 11:25:03 -04:00
< 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 >
2015-07-02 02:11:02 -04:00
< td > 1g< / 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.
2015-07-02 02:11:02 -04:00
(e.g. < code > 1g< / code > , < code > 2g< / code > ).
2015-09-21 14:46:39 -04:00
2015-02-25 19:12:56 -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-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 >
2015-07-22 18:28:09 -04:00
< td > 1g< / td >
2014-10-22 00:53:09 -04:00
< td >
2015-07-22 18:28:09 -04:00
Amount of memory to use per executor process (e.g. < code > 2g< / code > , < code > 8g< / 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
2016-01-21 10:30:20 -05:00
NOTE: In Spark 1.0 and later this will be overridden 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 >
2015-12-15 21:24:23 -05:00
< tr >
< td > < code > spark.submit.deployMode< / code > < / td >
< td > (none)< / td >
< td >
The deploy mode of Spark driver program, either "client" or "cluster",
Which means to launch driver program locally ("client")
or remotely ("cluster") on one of the nodes inside the cluster.
< / td >
< / tr >
2016-11-11 21:36:23 -05:00
< tr >
< td > < code > spark.log.callerContext< / code > < / td >
< td > (none)< / td >
< td >
Application information that will be written into Yarn RM log/HDFS audit log when running on Yarn/HDFS.
Its length depends on the Hadoop configuration < code > hadoop.caller.context.max.size< / code > . It should be concise,
and typically can have up to 50 characters.
< / 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:
2017-01-07 14:15:51 -05: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-07-28 14:48:56 -04:00
Extra classpath entries to prepend to the classpath of the driver.
2015-02-25 19:12:56 -05:00
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-09-21 14:46:39 -04:00
Instead, please set this through the < code > --driver-class-path< / code > command line option or in
2016-09-17 11:25:03 -04:00
your default properties file.
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.
2016-04-14 11:29:14 -04:00
Note that it is illegal to set maximum heap size (-Xmx) settings with this option. Maximum heap
size settings can be set with < code > spark.driver.memory< / code > in the cluster mode and through
the < code > --driver-memory< / code > command line option in the client mode.
2015-09-21 14:46:39 -04:00
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-09-21 14:46:39 -04:00
Instead, please set this through the < code > --driver-java-options< / code > command line option or in
2016-04-14 11:29:14 -04:00
your default properties file.
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-09-21 14:46:39 -04:00
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-09-21 14:46:39 -04:00
Instead, please set this through the < code > --driver-library-path< / code > command line option or in
2016-09-17 11:25:03 -04:00
your default properties file.
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
2016-02-21 18:27:07 -05:00
classes in the driver. This feature can be used to mitigate conflicts between Spark's
[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
dependencies and user dependencies. It is currently an experimental feature.
2015-09-21 14:46:39 -04:00
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-09-21 14:46:39 -04:00
Extra classpath entries to prepend to the classpath of executors. This exists primarily for
backwards-compatibility with older versions of Spark. Users typically should not need to set
2015-02-25 19:12:56 -05:00
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-09-21 14:46:39 -04:00
A string of extra JVM options to pass to executors. For instance, GC settings or other logging.
2016-04-14 11:29:14 -04:00
Note that it is illegal to set Spark properties or maximum heap size (-Xmx) 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. Maximum 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 >
2016-10-18 16:23:31 -04:00
< tr >
< td > < code > spark.executor.logs.rolling.enableCompression< / code > < / td >
< td > false< / td >
< td >
Enable executor log compression. If it is enabled, the rolled executor logs will be compressed.
Disabled by default.
< / td >
< / tr >
2015-02-25 19:12:56 -05:00
< 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 >
2016-05-09 14:02:13 -04:00
Set the max size of the file in bytes 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.
2016-05-09 14:02:13 -04:00
For "size", use < code > spark.executor.logs.rolling.maxSize< / code > to set
2014-12-11 09:21:23 -05:00
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.
2015-11-14 06:59:37 -05:00
Rolling is disabled by default. Valid values are < code > daily< / code > , < code > hourly< / code > , < code > minutely< / code > or
2014-12-11 09:21:23 -05:00
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-18535][UI][YARN] Redact sensitive information from Spark logs and UI
## What changes were proposed in this pull request?
This patch adds a new property called `spark.secret.redactionPattern` that
allows users to specify a scala regex to decide which Spark configuration
properties and environment variables in driver and executor environments
contain sensitive information. When this regex matches the property or
environment variable name, its value is redacted from the environment UI and
various logs like YARN and event logs.
This change uses this property to redact information from event logs and YARN
logs. It also, updates the UI code to adhere to this property instead of
hardcoding the logic to decipher which properties are sensitive.
Here's an image of the UI post-redaction:
![image](https://cloud.githubusercontent.com/assets/1709451/20506215/4cc30654-b007-11e6-8aee-4cde253fba2f.png)
Here's the text in the YARN logs, post-redaction:
``HADOOP_CREDSTORE_PASSWORD -> *********(redacted)``
Here's the text in the event logs, post-redaction:
``...,"spark.executorEnv.HADOOP_CREDSTORE_PASSWORD":"*********(redacted)","spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD":"*********(redacted)",...``
## How was this patch tested?
1. Unit tests are added to ensure that redaction works.
2. A YARN job reading data off of S3 with confidential information
(hadoop credential provider password) being provided in the environment
variables of driver and executor. And, afterwards, logs were grepped to make
sure that no mention of secret password was present. It was also ensure that
the job was able to read the data off of S3 correctly, thereby ensuring that
the sensitive information was being trickled down to the right places to read
the data.
3. The event logs were checked to make sure no mention of secret password was
present.
4. UI environment tab was checked to make sure there was no secret information
being displayed.
Author: Mark Grover <mark@apache.org>
Closes #15971 from markgrover/master_redaction.
2016-11-28 11:59:47 -05:00
< tr >
< td > < code > spark.redaction.regex< / code > < / td >
< td > (?i)secret|password< / td >
< td >
Regex to decide which Spark configuration properties and environment variables in driver and
executor environments contain sensitive information. When this regex matches a property, its
value is redacted from the environment UI and various logs like YARN and event logs.
< / 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 >
2015-11-14 06:59:37 -05:00
Enable profiling in Python worker, the profile result will show up by < code > sc.show_profiles()< / code > ,
[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
or it will be displayed before the driver exiting. It also can be dumped into disk by
2015-11-14 06:59:37 -05:00
< code > sc.dump_profiles(path)< / code > . If some of the profile results had been displayed manually,
[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
they will not be displayed automatically before driver exiting.
2015-01-28 16:48:06 -05:00
2015-11-14 06:59:37 -05:00
By default the < code > pyspark.profiler.BasicProfiler< / code > will be used, but this can be overridden by
passing a profiler class in as a parameter to the < code > SparkContext< / code > 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
2016-02-21 18:27:07 -05:00
if there is large broadcast, then the broadcast will not be needed to transferred
2014-09-13 19:22:04 -04:00
from JVM to Python worker for every task.
< / td >
< / tr >
2016-06-07 12:28:39 -04:00
< tr >
< td > < code > spark.files< / code > < / td >
< td > < / td >
< td >
Comma-separated list of files to be placed in the working directory of each executor.
< / td >
< / tr >
< tr >
< td > < code > spark.submit.pyFiles< / code > < / td >
< td > < / td >
< td >
Comma-separated list of .zip, .egg, or .py files to place on the PYTHONPATH for Python apps.
< / td >
< / tr >
< tr >
< td > < code > spark.jars< / code > < / td >
< td > < / td >
< td >
Comma-separated list of local jars to include on the driver and executor classpaths.
< / td >
< / tr >
< tr >
< td > < code > spark.jars.packages< / code > < / td >
< td > < / td >
< td >
2017-01-24 06:32:11 -05:00
Comma-separated list of Maven coordinates of jars to include on the driver and executor
2017-01-11 14:57:38 -05:00
classpaths. The coordinates should be groupId:artifactId:version. If < code > spark.jars.ivySettings< / code >
is given artifacts will be resolved according to the configuration in the file, otherwise artifacts
will be searched for in the local maven repo, then maven central and finally any additional remote
repositories given by the command-line option < code > --repositories< / code > . For more details, see
< a href = "submitting-applications.html#advanced-dependency-management" > Advanced Dependency Management< / a > .
2016-06-07 12:28:39 -04:00
< / td >
< / tr >
< tr >
< td > < code > spark.jars.excludes< / code > < / td >
< td > < / td >
< td >
Comma-separated list of groupId:artifactId, to exclude while resolving the dependencies
provided in < code > spark.jars.packages< / code > to avoid dependency conflicts.
< / td >
< / tr >
< tr >
< td > < code > spark.jars.ivy< / code > < / td >
< td > < / td >
< td >
2017-01-11 14:57:38 -05:00
Path to specify the Ivy user directory, used for the local Ivy cache and package files from
< code > spark.jars.packages< / code > . This will override the Ivy property < code > ivy.default.ivy.user.dir< / code >
which defaults to ~/.ivy2.
< / td >
< / tr >
< tr >
< td > < code > spark.jars.ivySettings< / code > < / td >
< td > < / td >
< td >
Path to an Ivy settings file to customize resolution of jars specified using < code > spark.jars.packages< / code >
instead of the built-in defaults, such as maven central. Additional repositories given by the command-line
option < code > --repositories< / code > will also be included. Useful for allowing Spark to resolve artifacts from behind
a firewall e.g. via an in-house artifact server like Artifactory. Details on the settings file format can be
found at http://ant.apache.org/ivy/history/latest-milestone/settings.html
2016-06-07 12:28:39 -04:00
< / td >
< / tr >
2016-08-11 23:08:25 -04:00
< tr >
< td > < code > spark.pyspark.driver.python< / code > < / td >
< td > < / td >
< td >
Python binary executable to use for PySpark in driver.
(default is < code > spark.pyspark.python< / code > )
< / td >
< / tr >
< tr >
< td > < code > spark.pyspark.python< / code > < / td >
< td > < / td >
< td >
Python binary executable to use for PySpark in both driver and executors.
< / td >
< / tr >
2015-02-25 19:12:56 -05:00
< / table >
2017-01-07 14:15:51 -05:00
### Shuffle Behavior
2015-02-25 19:12:56 -05:00
< 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 >
2016-02-12 01:40:00 -05:00
< tr >
< td > < code > spark.reducer.maxReqsInFlight< / code > < / td >
< td > Int.MaxValue< / td >
< td >
This configuration limits the number of remote requests to fetch blocks at any given point.
When the number of hosts in the cluster increase, it might lead to very large number
of in-bound connections to one or more nodes, causing the workers to fail under load.
By allowing it to limit the number of fetch requests, this scenario can be mitigated.
< / 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 >
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 >
2015-08-06 14:29:42 -04:00
< tr >
< td > < code > spark.shuffle.service.enabled< / code > < / td >
< td > false< / td >
< td >
2015-09-21 14:46:39 -04:00
Enables the external shuffle service. This service preserves the shuffle files written by
executors so the executors can be safely removed. This must be enabled if
2015-08-06 14:29:42 -04:00
< code > spark.dynamicAllocation.enabled< / code > is "true". The external shuffle service
must be set up in order to enable it. See
2015-09-21 14:46:39 -04:00
< a href = "job-scheduling.html#configuration-and-setup" > dynamic allocation
2015-08-06 14:29:42 -04:00
configuration and setup documentation< / a > for more information.
< / td >
< / tr >
< tr >
< td > < code > spark.shuffle.service.port< / code > < / td >
< td > 7337< / td >
< td >
Port on which the external shuffle service will run.
< / td >
< / tr >
2016-08-04 17:54:38 -04:00
< tr >
< td > < code > spark.shuffle.service.index.cache.entries< / code > < / td >
< td > 1024< / td >
< td >
Max number of entries to keep in the index cache of the shuffle service.
< / 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 >
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 >
2016-08-30 12:15:31 -04:00
< tr >
< td > < code > spark.io.encryption.enabled< / code > < / td >
< td > false< / td >
< td >
2016-11-29 00:10:57 -05:00
Enable IO encryption. Currently supported by all modes except Mesos. It's recommended that RPC encryption
be enabled when using this feature.
2016-08-30 12:15:31 -04:00
< / td >
< / tr >
< tr >
< td > < code > spark.io.encryption.keySizeBits< / code > < / td >
< td > 128< / td >
< td >
IO encryption key size in bits. Supported values are 128, 192 and 256.
< / td >
< / tr >
< tr >
< td > < code > spark.io.encryption.keygen.algorithm< / code > < / td >
< td > HmacSHA1< / td >
< td >
The algorithm to use when generating the IO encryption key. The supported algorithms are
described in the KeyGenerator section of the Java Cryptography Architecture Standard Algorithm
Name Documentation.
< / td >
< / tr >
2015-02-25 19:12:56 -05:00
< / table >
2017-01-07 14:15:51 -05:00
### 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.
2017-04-01 06:48:58 -04:00
Compression will use < code > spark.io.compression.codec< / code > .
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 >
2016-08-12 15:10:09 -04:00
< tr >
< td > < code > spark.ui.enabled< / code > < / td >
< td > true< / td >
< td >
Whether to run the web UI for the Spark application.
< / 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 >
2016-10-26 08:26:54 -04:00
Allows jobs and stages to be killed from the web UI.
2014-04-10 20:10:11 -04:00
< / 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 >
2017-01-23 06:02:22 -05:00
How many jobs the Spark UI and status APIs remember before garbage collecting.
This is a target maximum, and fewer elements may be retained in some circumstances.
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 >
2017-01-23 06:02:22 -05:00
How many stages the Spark UI and status APIs remember before garbage collecting.
This is a target maximum, and fewer elements may be retained in some circumstances.
2014-05-28 18:49:54 -04:00
< / td >
< / tr >
2016-08-24 15:39:41 -04:00
< tr >
< td > < code > spark.ui.retainedTasks< / code > < / td >
< td > 100000< / td >
< td >
2017-01-23 06:02:22 -05:00
How many tasks the Spark UI and status APIs remember before garbage collecting.
This is a target maximum, and fewer elements may be retained in some circumstances.
2016-08-24 15:39:41 -04:00
< / td >
< / tr >
2016-09-08 20:20:20 -04:00
< tr >
< td > < code > spark.ui.reverseProxy< / code > < / td >
< td > false< / td >
< td >
Enable running Spark Master as reverse proxy for worker and application UIs. In this mode, Spark master will reverse proxy the worker and application UIs to enable access without requiring direct access to their hosts. Use it with caution, as worker and application UI will not be accessible directly, you will only be able to access them through spark master/proxy public URL. This setting affects all the workers and application UIs running in the cluster and must be set on all the workers, drivers and masters.
< / td >
< / tr >
< tr >
< td > < code > spark.ui.reverseProxyUrl< / code > < / td >
< td > < / td >
< td >
This is the URL where your proxy is running. This URL is for proxy which is running in front of Spark Master. This is useful when running proxy for authentication e.g. OAuth proxy. Make sure this is a complete URL including scheme (http/https) and port to reach your proxy.
< / td >
< / tr >
2016-12-05 17:40:50 -05:00
< tr >
< td > < code > spark.ui.showConsoleProgress< / code > < / td >
< td > true< / td >
< td >
Show the progress bar in the console. The progress bar shows the progress of stages
that run for longer than 500ms. If multiple stages run at the same time, multiple
progress bars will be displayed on the same line.
< / td >
< / tr >
2015-07-31 15:27:00 -04:00
< tr >
< td > < code > spark.worker.ui.retainedExecutors< / code > < / td >
< td > 1000< / td >
< td >
How many finished executors the Spark UI and status APIs remember before garbage collecting.
< / td >
< / tr >
< tr >
< td > < code > spark.worker.ui.retainedDrivers< / code > < / td >
< td > 1000< / td >
< td >
How many finished drivers the Spark UI and status APIs remember before garbage collecting.
< / td >
< / tr >
2015-10-15 15:45:37 -04:00
< tr >
< td > < code > spark.sql.ui.retainedExecutions< / code > < / td >
< td > 1000< / td >
< td >
How many finished executions the Spark UI and status APIs remember before garbage collecting.
< / td >
< / tr >
< tr >
< td > < code > spark.streaming.ui.retainedBatches< / code > < / td >
< td > 1000< / td >
< td >
How many finished batches the Spark UI and status APIs remember before garbage collecting.
< / td >
< / tr >
2016-02-23 14:08:39 -05:00
< tr >
< td > < code > spark.ui.retainedDeadExecutors< / code > < / td >
< td > 100< / td >
< td >
How many dead executors the Spark UI and status APIs remember before garbage collecting.
< / td >
< / tr >
2014-05-28 18:49:54 -04:00
< / table >
2017-01-07 14:15:51 -05:00
### Compression and Serialization
2014-05-28 18:49:54 -04:00
< 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.
2017-04-01 06:48:58 -04:00
Compression will use < code > spark.io.compression.codec< / code > .
2012-09-27 21:45:44 -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 >
2015-12-21 17:21:43 -05:00
< td > lz4< / td >
2013-07-30 20:12:16 -04:00
< td >
2017-04-01 06:48:58 -04:00
The codec used to compress internal data such as RDD partitions, event log, 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 >
2016-11-01 19:23:47 -04:00
< td > true< / td >
2013-07-14 21:55:54 -04:00
< 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-12-04 19:58:31 -05:00
If you use Kryo serialization, give a comma-separated list of classes that register your custom classes with Kryo. This
2015-02-25 19:12:56 -05:00
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
2015-12-04 19:58:31 -05:00
set to classes that extend
2015-02-25 19:12:56 -05:00
< 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 >
2016-10-22 15:03:37 -04:00
< tr >
< td > < code > spark.kryo.unsafe< / code > < / td >
< td > false< / td >
< td >
Whether to use unsafe based Kryo serializer. Can be
substantially faster by using Unsafe Based IO.
< / td >
< / tr >
2014-07-30 03:18:59 -04:00
< 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
2016-12-27 15:29:45 -05:00
object you attempt to serialize and must be less than 2048m.
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
2015-07-14 03:54:30 -04:00
< code > spark.kryoserializer.buffer.max< / code > if needed.
2015-02-25 19:12:56 -05:00
< / td >
< / tr >
< tr >
< td > < code > spark.rdd.compress< / code > < / td >
< td > false< / td >
< td >
2016-01-21 10:30:20 -05:00
Whether to compress serialized RDD partitions (e.g. for
< code > StorageLevel.MEMORY_ONLY_SER< / code > in Java
and Scala or < code > StorageLevel.MEMORY_ONLY< / code > in Python).
Can save substantial space at the cost of some extra CPU time.
2017-04-01 06:48:58 -04:00
Compression will use < code > spark.io.compression.codec< / code > .
2015-02-25 19:12:56 -05:00
< / td >
< / tr >
< tr >
< td > < code > spark.serializer< / code > < / td >
2015-06-08 04:07:50 -04:00
< td >
2016-11-01 19:23:47 -04:00
org.apache.spark.serializer.< br / > JavaSerializer
2015-06-08 04:07:50 -04:00
< / td >
2015-02-25 19:12:56 -05:00
< 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 >
2017-01-07 14:15:51 -05:00
### Memory Management
2015-10-13 16:49:59 -04:00
< table class = "table" >
< tr > < th > Property Name< / th > < th > Default< / th > < th > Meaning< / th > < / tr >
< tr >
< td > < code > spark.memory.fraction< / code > < / td >
2016-06-16 17:04:10 -04:00
< td > 0.6< / td >
2015-10-13 16:49:59 -04:00
< td >
2015-12-01 22:51:12 -05:00
Fraction of (heap space - 300MB) used for execution and storage. The lower this is, the
more frequently spills and cached data eviction occur. The purpose of this config is to set
2015-10-13 16:49:59 -04:00
aside memory for internal metadata, user data structures, and imprecise size estimation
2015-11-16 20:00:18 -05:00
in the case of sparse, unusually large records. Leaving this at the default value is
2016-06-16 17:04:10 -04:00
recommended. For more detail, including important information about correctly tuning JVM
garbage collection when increasing this value, see
< a href = "tuning.html#memory-management-overview" > this description< / a > .
2015-10-13 16:49:59 -04:00
< / td >
< / tr >
< tr >
< td > < code > spark.memory.storageFraction< / code > < / td >
< td > 0.5< / td >
< td >
2015-11-16 20:00:18 -05:00
Amount of storage memory immune to eviction, expressed as a fraction of the size of the
region set aside by < code > s park.memory.fraction< / code > . The higher this is, the less
working memory may be available to execution and tasks may spill to disk more often.
Leaving this at the default value is recommended. For more detail, see
< a href = "tuning.html#memory-management-overview" > this description< / a > .
2015-10-13 16:49:59 -04:00
< / td >
< / tr >
2015-12-10 18:29:04 -05:00
< tr >
< td > < code > spark.memory.offHeap.enabled< / code > < / td >
2016-01-07 02:06:21 -05:00
< td > false< / td >
2015-12-10 18:29:04 -05:00
< td >
If true, Spark will attempt to use off-heap memory for certain operations. If off-heap memory use is enabled, then < code > spark.memory.offHeap.size< / code > must be positive.
< / td >
< / tr >
< tr >
< td > < code > spark.memory.offHeap.size< / code > < / td >
< td > 0< / td >
< td >
2016-03-07 15:07:50 -05:00
The absolute amount of memory in bytes which can be used for off-heap allocation.
2015-12-10 18:29:04 -05:00
This setting has no impact on heap memory usage, so if your executors' total memory consumption must fit within some hard limit then be sure to shrink your JVM heap size accordingly.
This must be set to a positive value when < code > spark.memory.offHeap.enabled=true< / code > .
< / td >
< / tr >
2015-10-13 16:49:59 -04:00
< tr >
< td > < code > spark.memory.useLegacyMode< / code > < / td >
< td > false< / td >
< td >
Whether to enable the legacy memory management mode used in Spark 1.5 and before.
The legacy mode rigidly partitions the heap space into fixed-size regions,
potentially leading to excessive spilling if the application was not tuned.
The following deprecated memory fraction configurations are not read unless this is enabled:
< code > spark.shuffle.memoryFraction< / code > < br >
< code > spark.storage.memoryFraction< / code > < br >
< code > spark.storage.unrollFraction< / code >
< / td >
< / tr >
< tr >
< td > < code > spark.shuffle.memoryFraction< / code > < / td >
< td > 0.2< / td >
< td >
(deprecated) This is read only if < code > spark.memory.useLegacyMode< / code > is enabled.
Fraction of Java heap to use for aggregation and cogroups during shuffles.
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 >
< tr >
< td > < code > spark.storage.memoryFraction< / code > < / td >
< td > 0.6< / td >
< td >
(deprecated) This is read only if < code > spark.memory.useLegacyMode< / code > is enabled.
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 >
< tr >
< td > < code > spark.storage.unrollFraction< / code > < / td >
< td > 0.2< / td >
< td >
(deprecated) This is read only if < code > spark.memory.useLegacyMode< / code > is enabled.
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 >
2017-02-24 18:40:01 -05:00
< tr >
< td > < code > spark.storage.replication.proactive< code > < / td >
< td > false< / td >
< td >
Enables proactive block replication for RDD blocks. Cached RDD block replicas lost due to
executor failures are replenished if there are any existing available replicas. This tries
to get the replication level of the block to the initial number.
< / td >
< / tr >
2015-10-13 16:49:59 -04:00
< / table >
2017-01-07 14:15:51 -05:00
### Execution Behavior
2014-05-28 18:49:54 -04:00
< 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 >
2015-04-14 16:32:06 -04:00
< tr >
< td > < code > spark.executor.cores< / code > < / td >
< td >
2016-02-10 13:53:33 -05:00
1 in YARN mode, all the available cores on the worker in
standalone and Mesos coarse-grained modes.
< / td >
< td >
The number of cores to use on each executor.
2015-09-21 14:46:39 -04:00
2016-02-10 13:53:33 -05:00
In standalone and Mesos coarse-grained modes, 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.
2015-04-14 16:32:06 -04:00
< / 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
2016-09-14 04:03:16 -04:00
tasks. spark.executor.heartbeatInterval should be significantly less than
spark.network.timeout< / 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 >
2016-11-07 21:41:17 -05:00
< tr >
< td > < code > spark.files.maxPartitionBytes< / code > < / td >
< td > 134217728 (128 MB)< / td >
< td >
The maximum number of bytes to pack into a single partition when reading files.
< / td >
< / tr >
< tr >
< td > < code > spark.files.openCostInBytes< / code > < / td >
< td > 4194304 (4 MB)< / td >
< td >
The estimated cost to open a file, measured by the number of bytes could be scanned in the same
time. This is used when putting multiple files into a partition. It is better to over estimate,
then the partitions with small files will be faster than partitions with bigger files.
< / td >
< / tr >
2013-08-15 20:22:49 -04:00
< 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 >
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 >
2017-03-30 05:39:57 -04:00
< tr >
< td > < code > spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version< / code > < / td >
< td > 1< / td >
< td >
The file output committer algorithm version, valid algorithm version number: 1 or 2.
Version 2 may have better performance, but version 1 may handle failures better in certain situations,
as per < a href = "https://issues.apache.org/jira/browse/MAPREDUCE-4815" > MAPREDUCE-4815< / a > .
< / td >
< / tr >
2014-05-28 18:49:54 -04:00
< / table >
2017-01-07 14:15:51 -05:00
### Networking
2014-05-28 18:49:54 -04:00
< table class = "table" >
< tr > < th > Property Name< / th > < th > Default< / th > < th > Meaning< / th > < / tr >
2013-01-21 16:10:02 -05:00
< tr >
2016-01-23 00:20:04 -05:00
< td > < code > spark.rpc.message.maxSize< / code > < / td >
2015-07-22 18:28:09 -04:00
< td > 128< / td >
2013-01-21 16:10:02 -05:00
< td >
2015-10-08 03:01:23 -04:00
Maximum message size (in MB) to allow in "control plane" communication; generally only applies to map
2015-07-22 18:28:09 -04:00
output size information sent between executors and the driver. Increase this if you are running
2016-01-23 00:20:04 -05:00
jobs with many thousands of map and reduce tasks and see messages about the RPC message size.
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 >
[SPARK-4563][CORE] Allow driver to advertise a different network address.
The goal of this feature is to allow the Spark driver to run in an
isolated environment, such as a docker container, and be able to use
the host's port forwarding mechanism to be able to accept connections
from the outside world.
The change is restricted to the driver: there is no support for achieving
the same thing on executors (or the YARN AM for that matter). Those still
need full access to the outside world so that, for example, connections
can be made to an executor's block manager.
The core of the change is simple: add a new configuration that tells what's
the address the driver should bind to, which can be different than the address
it advertises to executors (spark.driver.host). Everything else is plumbing
the new configuration where it's needed.
To use the feature, the host starting the container needs to set up the
driver's port range to fall into a range that is being forwarded; this
required the block manager port to need a special configuration just for
the driver, which falls back to the existing spark.blockManager.port when
not set. This way, users can modify the driver settings without affecting
the executors; it would theoretically be nice to also have different
retry counts for driver and executors, but given that docker (at least)
allows forwarding port ranges, we can probably live without that for now.
Because of the nature of the feature it's kinda hard to add unit tests;
I just added a simple one to make sure the configuration works.
This was tested with a docker image running spark-shell with the following
command:
docker blah blah blah \
-p 38000-38100:38000-38100 \
[image] \
spark-shell \
--num-executors 3 \
--conf spark.shuffle.service.enabled=false \
--conf spark.dynamicAllocation.enabled=false \
--conf spark.driver.host=[host's address] \
--conf spark.driver.port=38000 \
--conf spark.driver.blockManager.port=38020 \
--conf spark.ui.port=38040
Running on YARN; verified the driver works, executors start up and listen
on ephemeral ports (instead of using the driver's config), and that caching
and shuffling (without the shuffle service) works. Clicked through the UI
to make sure all pages (including executor thread dumps) worked. Also tested
apps without docker, and ran unit tests.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #15120 from vanzin/SPARK-4563.
2016-09-21 17:42:41 -04:00
< tr >
< td > < code > spark.driver.blockManager.port< / code > < / td >
< td > (value of spark.blockManager.port)< / td >
< td >
Driver-specific port for the block manager to listen on, for cases where it cannot use the same
configuration as executors.
< / td >
< / tr >
< tr >
< td > < code > spark.driver.bindAddress< / code > < / td >
< td > (value of spark.driver.host)< / td >
< td >
2017-01-07 14:15:51 -05:00
Hostname or IP address where to bind listening sockets. This config overrides the SPARK_LOCAL_IP
environment variable (see below).
[SPARK-4563][CORE] Allow driver to advertise a different network address.
The goal of this feature is to allow the Spark driver to run in an
isolated environment, such as a docker container, and be able to use
the host's port forwarding mechanism to be able to accept connections
from the outside world.
The change is restricted to the driver: there is no support for achieving
the same thing on executors (or the YARN AM for that matter). Those still
need full access to the outside world so that, for example, connections
can be made to an executor's block manager.
The core of the change is simple: add a new configuration that tells what's
the address the driver should bind to, which can be different than the address
it advertises to executors (spark.driver.host). Everything else is plumbing
the new configuration where it's needed.
To use the feature, the host starting the container needs to set up the
driver's port range to fall into a range that is being forwarded; this
required the block manager port to need a special configuration just for
the driver, which falls back to the existing spark.blockManager.port when
not set. This way, users can modify the driver settings without affecting
the executors; it would theoretically be nice to also have different
retry counts for driver and executors, but given that docker (at least)
allows forwarding port ranges, we can probably live without that for now.
Because of the nature of the feature it's kinda hard to add unit tests;
I just added a simple one to make sure the configuration works.
This was tested with a docker image running spark-shell with the following
command:
docker blah blah blah \
-p 38000-38100:38000-38100 \
[image] \
spark-shell \
--num-executors 3 \
--conf spark.shuffle.service.enabled=false \
--conf spark.dynamicAllocation.enabled=false \
--conf spark.driver.host=[host's address] \
--conf spark.driver.port=38000 \
--conf spark.driver.blockManager.port=38020 \
--conf spark.ui.port=38040
Running on YARN; verified the driver works, executors start up and listen
on ephemeral ports (instead of using the driver's config), and that caching
and shuffling (without the shuffle service) works. Clicked through the UI
to make sure all pages (including executor thread dumps) worked. Also tested
apps without docker, and ran unit tests.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #15120 from vanzin/SPARK-4563.
2016-09-21 17:42:41 -04:00
2017-01-07 14:15:51 -05:00
< br / > It also allows a different address from the local one to be advertised to executors or external systems.
[SPARK-4563][CORE] Allow driver to advertise a different network address.
The goal of this feature is to allow the Spark driver to run in an
isolated environment, such as a docker container, and be able to use
the host's port forwarding mechanism to be able to accept connections
from the outside world.
The change is restricted to the driver: there is no support for achieving
the same thing on executors (or the YARN AM for that matter). Those still
need full access to the outside world so that, for example, connections
can be made to an executor's block manager.
The core of the change is simple: add a new configuration that tells what's
the address the driver should bind to, which can be different than the address
it advertises to executors (spark.driver.host). Everything else is plumbing
the new configuration where it's needed.
To use the feature, the host starting the container needs to set up the
driver's port range to fall into a range that is being forwarded; this
required the block manager port to need a special configuration just for
the driver, which falls back to the existing spark.blockManager.port when
not set. This way, users can modify the driver settings without affecting
the executors; it would theoretically be nice to also have different
retry counts for driver and executors, but given that docker (at least)
allows forwarding port ranges, we can probably live without that for now.
Because of the nature of the feature it's kinda hard to add unit tests;
I just added a simple one to make sure the configuration works.
This was tested with a docker image running spark-shell with the following
command:
docker blah blah blah \
-p 38000-38100:38000-38100 \
[image] \
spark-shell \
--num-executors 3 \
--conf spark.shuffle.service.enabled=false \
--conf spark.dynamicAllocation.enabled=false \
--conf spark.driver.host=[host's address] \
--conf spark.driver.port=38000 \
--conf spark.driver.blockManager.port=38020 \
--conf spark.ui.port=38040
Running on YARN; verified the driver works, executors start up and listen
on ephemeral ports (instead of using the driver's config), and that caching
and shuffling (without the shuffle service) works. Clicked through the UI
to make sure all pages (including executor thread dumps) worked. Also tested
apps without docker, and ran unit tests.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #15120 from vanzin/SPARK-4563.
2016-09-21 17:42:41 -04:00
This is useful, for example, when running containers with bridged networking. For this to properly work,
the different ports used by the driver (RPC, block manager and UI) need to be forwarded from the
2017-01-07 14:15:51 -05:00
container's host.
[SPARK-4563][CORE] Allow driver to advertise a different network address.
The goal of this feature is to allow the Spark driver to run in an
isolated environment, such as a docker container, and be able to use
the host's port forwarding mechanism to be able to accept connections
from the outside world.
The change is restricted to the driver: there is no support for achieving
the same thing on executors (or the YARN AM for that matter). Those still
need full access to the outside world so that, for example, connections
can be made to an executor's block manager.
The core of the change is simple: add a new configuration that tells what's
the address the driver should bind to, which can be different than the address
it advertises to executors (spark.driver.host). Everything else is plumbing
the new configuration where it's needed.
To use the feature, the host starting the container needs to set up the
driver's port range to fall into a range that is being forwarded; this
required the block manager port to need a special configuration just for
the driver, which falls back to the existing spark.blockManager.port when
not set. This way, users can modify the driver settings without affecting
the executors; it would theoretically be nice to also have different
retry counts for driver and executors, but given that docker (at least)
allows forwarding port ranges, we can probably live without that for now.
Because of the nature of the feature it's kinda hard to add unit tests;
I just added a simple one to make sure the configuration works.
This was tested with a docker image running spark-shell with the following
command:
docker blah blah blah \
-p 38000-38100:38000-38100 \
[image] \
spark-shell \
--num-executors 3 \
--conf spark.shuffle.service.enabled=false \
--conf spark.dynamicAllocation.enabled=false \
--conf spark.driver.host=[host's address] \
--conf spark.driver.port=38000 \
--conf spark.driver.blockManager.port=38020 \
--conf spark.ui.port=38040
Running on YARN; verified the driver works, executors start up and listen
on ephemeral ports (instead of using the driver's config), and that caching
and shuffling (without the shuffle service) works. Clicked through the UI
to make sure all pages (including executor thread dumps) worked. Also tested
apps without docker, and ran unit tests.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #15120 from vanzin/SPARK-4563.
2016-09-21 17:42:41 -04: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 >
[SPARK-4563][CORE] Allow driver to advertise a different network address.
The goal of this feature is to allow the Spark driver to run in an
isolated environment, such as a docker container, and be able to use
the host's port forwarding mechanism to be able to accept connections
from the outside world.
The change is restricted to the driver: there is no support for achieving
the same thing on executors (or the YARN AM for that matter). Those still
need full access to the outside world so that, for example, connections
can be made to an executor's block manager.
The core of the change is simple: add a new configuration that tells what's
the address the driver should bind to, which can be different than the address
it advertises to executors (spark.driver.host). Everything else is plumbing
the new configuration where it's needed.
To use the feature, the host starting the container needs to set up the
driver's port range to fall into a range that is being forwarded; this
required the block manager port to need a special configuration just for
the driver, which falls back to the existing spark.blockManager.port when
not set. This way, users can modify the driver settings without affecting
the executors; it would theoretically be nice to also have different
retry counts for driver and executors, but given that docker (at least)
allows forwarding port ranges, we can probably live without that for now.
Because of the nature of the feature it's kinda hard to add unit tests;
I just added a simple one to make sure the configuration works.
This was tested with a docker image running spark-shell with the following
command:
docker blah blah blah \
-p 38000-38100:38000-38100 \
[image] \
spark-shell \
--num-executors 3 \
--conf spark.shuffle.service.enabled=false \
--conf spark.dynamicAllocation.enabled=false \
--conf spark.driver.host=[host's address] \
--conf spark.driver.port=38000 \
--conf spark.driver.blockManager.port=38020 \
--conf spark.ui.port=38040
Running on YARN; verified the driver works, executors start up and listen
on ephemeral ports (instead of using the driver's config), and that caching
and shuffling (without the shuffle service) works. Clicked through the UI
to make sure all pages (including executor thread dumps) worked. Also tested
apps without docker, and ran unit tests.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #15120 from vanzin/SPARK-4563.
2016-09-21 17:42:41 -04:00
Hostname or IP address for the driver.
2015-02-25 19:12:56 -05:00
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 >
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-09-21 14:46:39 -04:00
Default timeout for all network interactions. This config will be used in place of
2016-01-23 00:20:04 -05:00
< code > spark.core.connection.ack.wait.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-09-03 16:46:16 -04:00
Maximum number of retries when binding to a port before giving up.
When a port is given a specific value (non 0), each subsequent retry will
2015-09-21 14:46:39 -04:00
increment the port used in the previous attempt by 1 before retrying. This
essentially allows it to try a range of ports from the start port specified
2015-09-03 16:46:16 -04:00
to port + maxRetries.
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 >
2015-07-08 22:28:51 -04:00
< td >
2015-04-21 21:37:53 -04:00
Number of times to retry before an RPC task gives up.
An RPC task will run at most times of this number.
< / 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 >
2016-11-19 06:28:25 -05:00
< td > < code > spark.network.timeout< / code > < / td >
2015-04-21 21:37:53 -04:00
< 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 >
< td >
2015-12-15 21:24:23 -05:00
Duration for an RPC remote endpoint lookup operation to wait before timing out.
2015-04-21 21:37:53 -04:00
< / td >
< / tr >
2015-02-25 19:12:56 -05:00
< / table >
2017-01-07 14:15:51 -05:00
### Scheduling
2015-02-25 19:12:56 -05:00
< 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 >
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-09-10 15:03:11 -04:00
< td > 0.8 for YARN mode; 0.0 for standalone mode and Mesos coarse-grained mode< / 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)
2015-09-10 15:03:11 -04:00
(resources are executors in yarn mode, CPU cores in standalone mode and Mesos coarsed-grained
mode ['spark.cores.max' value is total expected resources for Mesos coarse-grained 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-17675][CORE] Expand Blacklist for TaskSets
## What changes were proposed in this pull request?
This is a step along the way to SPARK-8425.
To enable incremental review, the first step proposed here is to expand the blacklisting within tasksets. In particular, this will enable blacklisting for
* (task, executor) pairs (this already exists via an undocumented config)
* (task, node)
* (taskset, executor)
* (taskset, node)
Adding (task, node) is critical to making spark fault-tolerant of one-bad disk in a cluster, without requiring careful tuning of "spark.task.maxFailures". The other additions are also important to avoid many misleading task failures and long scheduling delays when there is one bad node on a large cluster.
Note that some of the code changes here aren't really required for just this -- they put pieces in place for SPARK-8425 even though they are not used yet (eg. the `BlacklistTracker` helper is a little out of place, `TaskSetBlacklist` holds onto a little more info than it needs to for just this change, and `ExecutorFailuresInTaskSet` is more complex than it needs to be).
## How was this patch tested?
Added unit tests, run tests via jenkins.
Author: Imran Rashid <irashid@cloudera.com>
Author: mwws <wei.mao@intel.com>
Closes #15249 from squito/taskset_blacklist_only.
2016-10-12 17:43:03 -04:00
< tr >
< td > < code > spark.blacklist.enabled< / code > < / td >
< td >
false
< / td >
< td >
If set to "true", prevent Spark from scheduling tasks on executors that have been blacklisted
due to too many task failures. The blacklisting algorithm can be further controlled by the
other "spark.blacklist" configuration options.
< / td >
< / tr >
2016-12-15 09:29:56 -05:00
< tr >
< td > < code > spark.blacklist.timeout< / code > < / td >
< td > 1h< / td >
< td >
(Experimental) How long a node or executor is blacklisted for the entire application, before it
is unconditionally removed from the blacklist to attempt running new tasks.
< / td >
< / tr >
[SPARK-17675][CORE] Expand Blacklist for TaskSets
## What changes were proposed in this pull request?
This is a step along the way to SPARK-8425.
To enable incremental review, the first step proposed here is to expand the blacklisting within tasksets. In particular, this will enable blacklisting for
* (task, executor) pairs (this already exists via an undocumented config)
* (task, node)
* (taskset, executor)
* (taskset, node)
Adding (task, node) is critical to making spark fault-tolerant of one-bad disk in a cluster, without requiring careful tuning of "spark.task.maxFailures". The other additions are also important to avoid many misleading task failures and long scheduling delays when there is one bad node on a large cluster.
Note that some of the code changes here aren't really required for just this -- they put pieces in place for SPARK-8425 even though they are not used yet (eg. the `BlacklistTracker` helper is a little out of place, `TaskSetBlacklist` holds onto a little more info than it needs to for just this change, and `ExecutorFailuresInTaskSet` is more complex than it needs to be).
## How was this patch tested?
Added unit tests, run tests via jenkins.
Author: Imran Rashid <irashid@cloudera.com>
Author: mwws <wei.mao@intel.com>
Closes #15249 from squito/taskset_blacklist_only.
2016-10-12 17:43:03 -04:00
< tr >
< td > < code > spark.blacklist.task.maxTaskAttemptsPerExecutor< / code > < / td >
< td > 1< / td >
< td >
(Experimental) For a given task, how many times it can be retried on one executor before the
executor is blacklisted for that task.
< / td >
< / tr >
< tr >
< td > < code > spark.blacklist.task.maxTaskAttemptsPerNode< / code > < / td >
< td > 2< / td >
< td >
(Experimental) For a given task, how many times it can be retried on one node, before the entire
node is blacklisted for that task.
< / td >
< / tr >
< tr >
2016-12-18 04:02:04 -05:00
< td > < code > spark.blacklist.stage.maxFailedTasksPerExecutor< / code > < / td >
[SPARK-17675][CORE] Expand Blacklist for TaskSets
## What changes were proposed in this pull request?
This is a step along the way to SPARK-8425.
To enable incremental review, the first step proposed here is to expand the blacklisting within tasksets. In particular, this will enable blacklisting for
* (task, executor) pairs (this already exists via an undocumented config)
* (task, node)
* (taskset, executor)
* (taskset, node)
Adding (task, node) is critical to making spark fault-tolerant of one-bad disk in a cluster, without requiring careful tuning of "spark.task.maxFailures". The other additions are also important to avoid many misleading task failures and long scheduling delays when there is one bad node on a large cluster.
Note that some of the code changes here aren't really required for just this -- they put pieces in place for SPARK-8425 even though they are not used yet (eg. the `BlacklistTracker` helper is a little out of place, `TaskSetBlacklist` holds onto a little more info than it needs to for just this change, and `ExecutorFailuresInTaskSet` is more complex than it needs to be).
## How was this patch tested?
Added unit tests, run tests via jenkins.
Author: Imran Rashid <irashid@cloudera.com>
Author: mwws <wei.mao@intel.com>
Closes #15249 from squito/taskset_blacklist_only.
2016-10-12 17:43:03 -04:00
< td > 2< / td >
< td >
(Experimental) How many different tasks must fail on one executor, within one stage, before the
executor is blacklisted for that stage.
< / td >
< / tr >
< tr >
< td > < code > spark.blacklist.stage.maxFailedExecutorsPerNode< / code > < / td >
< td > 2< / td >
< td >
(Experimental) How many different executors are marked as blacklisted for a given stage, before
the entire node is marked as failed for the stage.
< / td >
< / tr >
2016-12-15 09:29:56 -05:00
< tr >
< td > < code > spark.blacklist.application.maxFailedTasksPerExecutor< / code > < / td >
< td > 2< / td >
< td >
(Experimental) How many different tasks must fail on one executor, in successful task sets,
before the executor is blacklisted for the entire application. Blacklisted executors will
be automatically added back to the pool of available resources after the timeout specified by
< code > spark.blacklist.timeout< / code > . Note that with dynamic allocation, though, the executors
may get marked as idle and be reclaimed by the cluster manager.
< / td >
< / tr >
< tr >
< td > < code > spark.blacklist.application.maxFailedExecutorsPerNode< / code > < / td >
< td > 2< / td >
< td >
(Experimental) How many different executors must be blacklisted for the entire application,
before the node is blacklisted for the entire application. Blacklisted nodes will
be automatically added back to the pool of available resources after the timeout specified by
< code > spark.blacklist.timeout< / code > . Note that with dynamic allocation, though, the executors
on the node may get marked as idle and be reclaimed by the cluster manager.
< / td >
< / tr >
2017-02-09 13:49:31 -05:00
< tr >
< td > < code > spark.blacklist.killBlacklistedExecutors< / code > < / td >
< td > false< / td >
< td >
(Experimental) If set to "true", allow Spark to automatically kill, and attempt to re-create,
executors when they are blacklisted. Note that, when an entire node is added to the blacklist,
all of the executors on that node will be killed.
< / 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 >
2016-07-25 22:18:27 -04:00
Fraction of tasks which must be complete before speculation is enabled for a particular stage.
2015-02-25 19:12:56 -05:00
< / 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 >
2016-07-22 07:41:38 -04:00
Number of failures of any particular task before giving up on the job.
The total number of failures spread across different tasks will not cause the job
to fail; a particular task has to fail this number of attempts.
2015-02-25 19:12:56 -05:00
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 >
2016-12-19 21:43:59 -05:00
< tr >
< td > < code > spark.task.reaper.enabled< / code > < / td >
< td > false< / td >
< td >
Enables monitoring of killed / interrupted tasks. When set to true, any task which is killed
will be monitored by the executor until that task actually finishes executing. See the other
< code > spark.task.reaper.*< / code > configurations for details on how to control the exact behavior
2017-01-11 14:57:38 -05:00
of this monitoring. When set to false (the default), task killing will use an older code
2016-12-19 21:43:59 -05:00
path which lacks such monitoring.
< / td >
< / tr >
< tr >
< td > < code > spark.task.reaper.pollingInterval< / code > < / td >
< td > 10s< / td >
< td >
When < code > spark.task.reaper.enabled = true< / code > , this setting controls the frequency at which
executors will poll the status of killed tasks. If a killed task is still running when polled
then a warning will be logged and, by default, a thread-dump of the task will be logged
(this thread dump can be disabled via the < code > spark.task.reaper.threadDump< / code > setting,
which is documented below).
< / td >
< / tr >
< tr >
< td > < code > spark.task.reaper.threadDump< / code > < / td >
< td > true< / td >
< td >
When < code > spark.task.reaper.enabled = true< / code > , this setting controls whether task thread
dumps are logged during periodic polling of killed tasks. Set this to false to disable
collection of thread dumps.
< / td >
< / tr >
< tr >
< td > < code > spark.task.reaper.killTimeout< / code > < / td >
< td > -1< / td >
< td >
When < code > spark.task.reaper.enabled = true< / code > , this setting specifies a timeout after
which the executor JVM will kill itself if a killed task has not stopped running. The default
value, -1, disables this mechanism and prevents the executor from self-destructing. The purpose
of this setting is to act as a safety-net to prevent runaway uncancellable tasks from rendering
an executor unusable.
< / td >
2017-03-17 10:33:45 -04:00
< td > < code > spark.stage.maxConsecutiveAttempts< / code > < / td >
< td > 4< / td >
< td >
Number of consecutive stage attempts allowed before a stage is aborted.
< / td >
2016-12-19 21:43:59 -05:00
< / tr >
2014-05-28 18:49:54 -04:00
< / table >
2017-01-07 14:15:51 -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
2016-08-08 19:07:51 -04:00
with this application up and down based on the workload.
2016-02-05 17:35:39 -05:00
For more detail, see the description
2014-12-19 22:36:20 -05:00
< 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-05-22 20:37:38 -04:00
< td > 60s< / td >
2015-02-25 19:12:56 -05:00
< td >
2015-09-21 14:46:39 -04:00
If dynamic allocation is enabled and an executor has been idle for more than this duration,
2015-04-13 19:28:07 -04:00
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 >
2015-06-07 00:13:26 -04:00
< tr >
< td > < code > spark.dynamicAllocation.cachedExecutorIdleTimeout< / code > < / td >
2015-07-10 12:48:17 -04:00
< td > infinity< / td >
2015-06-07 00:13:26 -04:00
< td >
If dynamic allocation is enabled and an executor which has cached data blocks has been idle for more than this duration,
the executor will be removed. For more details, see this
< a href = "job-scheduling.html#resource-allocation-policy" > description< / a > .
< / td >
< / tr >
2015-02-25 19:12:56 -05:00
< 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.
2016-06-23 15:03:46 -04:00
< br / > < br / >
If `--num-executors` (or `spark.executor.instances` ) is set and larger than this value, it will
be used as the initial number of executors.
2014-12-19 22:36:20 -05:00
< / td >
< / tr >
< tr >
< td > < code > spark.dynamicAllocation.maxExecutors< / code > < / td >
2015-07-10 12:48:17 -04:00
< td > infinity< / td >
2015-02-02 15:27:08 -05:00
< 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-05-22 20:37:38 -04:00
< td > 1s< / 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 >
2017-01-07 14:15:51 -05:00
### Security
2014-05-28 18:49:54 -04:00
< 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 >
2016-05-04 09:45:43 -04:00
Whether Spark acls should be enabled. If enabled, this checks to see if the user has
2015-02-25 19:12:56 -05:00
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
2016-05-04 09:45:43 -04:00
help debug when things do not work. Putting a "*" in the list means any user can have the
privilege of admin.
< / td >
< / tr >
< tr >
< td > < code > spark.admin.acls.groups< / code > < / td >
< td > Empty< / td >
< td >
Comma separated list of groups that have view and modify access to all Spark jobs.
This can be used if you have a set of administrators or developers who help maintain and debug
the underlying infrastructure. Putting a "*" in the list means any user in any group can have
the privilege of admin. The user groups are obtained from the instance of the groups mapping
provider specified by < code > spark.user.groups.mapping< / code > . Check the entry
< code > spark.user.groups.mapping< / code > for more details.
< / td >
< / tr >
< tr >
< td > < code > spark.user.groups.mapping< / code > < / td >
< td > < code > org.apache.spark.security.ShellBasedGroupsMappingProvider< / code > < / td >
< td >
The list of groups for a user are determined by a group mapping service defined by the trait
org.apache.spark.security.GroupMappingServiceProvider which can configured by this property.
A default unix shell based implementation is provided < code > org.apache.spark.security.ShellBasedGroupsMappingProvider< / code >
which can be specified to resolve a list of groups for a user.
< em > Note:< / em > This implementation supports only a Unix/Linux based environment. Windows environment is
currently < b > not< / b > supported. However, a new platform/protocol can be supported by implementing
the trait < code > org.apache.spark.security.GroupMappingServiceProvider< / code > .
2015-02-25 19:12:56 -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< / 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-09-21 16:15:44 -04:00
< tr >
2017-01-24 13:44:04 -05:00
< td > < code > spark.network.crypto.enabled< / code > < / td >
2015-09-21 16:15:44 -04:00
< td > false< / td >
< td >
2017-01-24 13:44:04 -05:00
Enable encryption using the commons-crypto library for RPC and block transfer service.
Requires < code > spark.authenticate< / code > to be enabled.
2015-09-21 16:15:44 -04:00
< / td >
< / tr >
< tr >
2017-01-24 13:44:04 -05:00
< td > < code > spark.network.crypto.keyLength< / code > < / td >
< td > 128< / td >
2015-09-21 16:15:44 -04:00
< td >
2017-01-24 13:44:04 -05:00
The length in bits of the encryption key to generate. Valid values are 128, 192 and 256.
2015-09-21 16:15:44 -04:00
< / td >
< / tr >
2016-11-11 13:37:58 -05:00
< tr >
2017-01-24 13:44:04 -05:00
< td > < code > spark.network.crypto.keyFactoryAlgorithm< / code > < / td >
< td > PBKDF2WithHmacSHA1< / td >
2016-11-11 13:37:58 -05:00
< td >
2017-01-24 13:44:04 -05:00
The key factory algorithm to use when generating encryption keys. Should be one of the
algorithms supported by the javax.crypto.SecretKeyFactory class in the JRE being used.
2016-11-11 13:37:58 -05:00
< / td >
< / tr >
< tr >
2017-01-24 13:44:04 -05:00
< td > < code > spark.network.crypto.saslFallback< / code > < / td >
< td > true< / td >
2016-11-11 13:37:58 -05:00
< td >
2017-01-24 13:44:04 -05:00
Whether to fall back to SASL authentication if authentication fails using Spark's internal
mechanism. This is useful when the application is connecting to old shuffle services that
do not support the internal Spark authentication protocol. On the server side, this can be
used to block older clients from authenticating against a new shuffle service.
2016-11-11 13:37:58 -05:00
< / td >
< / tr >
< tr >
2017-01-24 13:44:04 -05:00
< td > < code > spark.network.crypto.config.*< / code > < / td >
2016-12-12 19:27:04 -05:00
< td > None< / td >
2016-11-11 13:37:58 -05:00
< td >
2016-12-12 19:27:04 -05:00
Configuration values for the commons-crypto library, such as which cipher implementations to
use. The config name should be the name of commons-crypto configuration without the
"commons.crypto" prefix.
2016-11-11 13:37:58 -05:00
< / td >
< / tr >
2017-01-24 13:44:04 -05:00
< tr >
< td > < code > spark.authenticate.enableSaslEncryption< / code > < / td >
< td > false< / td >
< td >
Enable encrypted communication when authentication is
enabled. This is supported by the block transfer service and the
RPC endpoints.
< / td >
< / tr >
< tr >
< td > < code > spark.network.sasl.serverAlwaysEncrypt< / code > < / td >
< td > false< / td >
< td >
Disable unencrypted connections for services that support SASL authentication.
< / td >
< / tr >
2015-02-25 19:12:56 -05:00
< tr >
< td > < code > spark.core.connection.ack.wait.timeout< / code > < / td >
2016-11-19 06:28:25 -05:00
< td > < code > spark.network.timeout< / code > < / 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
2015-09-01 17:14:59 -04:00
user that started the Spark job has access to modify it (kill it for example). Putting a "*" in
the list means any user can have access to modify it.
2014-08-16 17:15:58 -04:00
< / td >
< / tr >
2016-05-04 09:45:43 -04:00
< tr >
< td > < code > spark.modify.acls.groups< / code > < / td >
< td > Empty< / td >
< td >
Comma separated list of groups that have modify access to the Spark job. This can be used if you
have a set of administrators or developers from the same team to have access to control the job.
Putting a "*" in the list means any user in any group has the access to modify the Spark job.
The user groups are obtained from the instance of the groups mapping provider specified by
< code > spark.user.groups.mapping< / code > . Check the entry < code > spark.user.groups.mapping< / code >
for more details.
< / 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
2015-09-01 17:14:59 -04:00
user that started the Spark job has view access. Putting a "*" in the list means any user can
have view access to this Spark job.
2014-04-21 13:26:33 -04:00
< / td >
< / tr >
2016-05-04 09:45:43 -04:00
< tr >
< td > < code > spark.ui.view.acls.groups< / code > < / td >
< td > Empty< / td >
< td >
Comma separated list of groups that have view access to the Spark web ui to view the Spark Job
details. This can be used if you have a set of administrators or developers or users who can
monitor the Spark job submitted. Putting a "*" in the list means any user in any group can view
the Spark job details on the Spark web ui. The user groups are obtained from the instance of the
groups mapping provider specified by < code > spark.user.groups.mapping< / code > . Check the entry
< code > spark.user.groups.mapping< / code > for more details.
< / td >
< / tr >
2014-05-28 18:49:54 -04:00
< / table >
2017-01-07 14:15:51 -05:00
### TLS / SSL
2015-02-02 20:18:54 -05:00
< 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 >
2017-01-07 14:15:51 -05:00
Whether to enable SSL connections on all supported protocols.
2015-02-02 20:18:54 -05:00
2017-01-07 14:15:51 -05:00
< br / > When < code > spark.ssl.enabled< / code > is configured, < code > spark.ssl.protocol< / code >
is required.
2016-08-21 06:51:46 -04:00
2017-01-07 14:15:51 -05:00
< br / > All the SSL settings like < code > spark.ssl.xxx< / code > where < code > xxx< / code > is a
2015-02-02 20:18:54 -05:00
particular configuration property, denote the global configuration for all the supported
protocols. In order to override the global configuration for the particular protocol,
2017-01-07 14:15:51 -05:00
the properties must be overwritten in the protocol-specific namespace.
2015-02-02 20:18:54 -05:00
2017-01-07 14:15:51 -05:00
< br / > Use < code > spark.ssl.YYY.XXX< / code > settings to overwrite the global configuration for
2016-08-11 06:26:57 -04:00
particular protocol denoted by < code > YYY< / code > . Example values for < code > YYY< / code >
include < code > fs< / code > , < code > ui< / code > , < code > standalone< / code > , and
< code > historyServer< / code > . See < a href = "security.html#ssl-configuration" > SSL
2017-01-07 14:15:51 -05:00
Configuration< / a > for details on hierarchical SSL configuration for services.
2015-02-02 20:18:54 -05:00
< / td >
< / tr >
2017-02-09 08:06:46 -05:00
< tr >
< td > < code > spark.ssl.[namespace].port< / code > < / td >
< td > None< / td >
< td >
The port where the SSL service will listen on.
< br / > The port must be defined within a namespace configuration; see
< a href = "security.html#ssl-configuration" > SSL Configuration< / a > for the available
namespaces.
< br / > When not set, the SSL port will be derived from the non-SSL port for the
same service. A value of "0" will make the service bind to an ephemeral port.
< / 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.
2016-01-19 17:49:55 -05:00
Note: If not set, it will use the default cipher suites of JVM.
2015-02-25 19:12:56 -05:00
< / 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 >
2016-01-19 17:49:55 -05:00
< tr >
< td > < code > spark.ssl.keyStoreType< / code > < / td >
< td > JKS< / td >
< td >
The type of the key-store.
< / td >
< / tr >
2015-02-02 20:18:54 -05:00
< 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 >
2016-01-19 17:49:55 -05:00
< tr >
< td > < code > spark.ssl.needClientAuth< / code > < / td >
< td > false< / td >
< td >
Set true if SSL needs client authentication.
< / td >
< / tr >
2015-02-02 20:18:54 -05:00
< 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 >
2016-01-19 17:49:55 -05:00
< tr >
< td > < code > spark.ssl.trustStoreType< / code > < / td >
< td > JKS< / td >
< td >
The type of the trust-store.
< / td >
< / tr >
2015-02-02 20:18:54 -05:00
< / table >
2017-01-07 14:15:51 -05:00
### Spark SQL
2016-05-24 00:07:14 -04:00
Running the < code > SET -v< / code > command will show the entire list of the SQL configuration.
< div class = "codetabs" >
< div data-lang = "scala" markdown = "1" >
{% highlight scala %}
// spark is an existing SparkSession
spark.sql("SET -v").show(numRows = 200, truncate = false)
{% endhighlight %}
< / div >
< div data-lang = "java" markdown = "1" >
{% highlight java %}
// spark is an existing SparkSession
spark.sql("SET -v").show(200, false);
{% endhighlight %}
< / div >
< div data-lang = "python" markdown = "1" >
{% highlight python %}
# spark is an existing SparkSession
spark.sql("SET -v").show(n=200, truncate=False)
{% endhighlight %}
< / div >
< div data-lang = "r" markdown = "1" >
{% highlight r %}
2016-07-06 13:41:48 -04:00
sparkR.session()
properties < - sql ( " SET -v " )
2016-05-24 00:07:14 -04:00
showDF(properties, numRows = 200, truncate = FALSE)
{% endhighlight %}
< / div >
< / div >
2017-01-07 14:15:51 -05:00
### Spark Streaming
2014-05-28 18:49:54 -04:00
< table class = "table" >
< tr > < th > Property Name< / th > < th > Default< / th > < th > Meaning< / th > < / tr >
2015-09-08 17:54:43 -04:00
< tr >
< td > < code > spark.streaming.backpressure.enabled< / code > < / td >
< td > false< / td >
< td >
Enables or disables Spark Streaming's internal backpressure mechanism (since 1.5).
2015-09-21 14:46:39 -04:00
This enables the Spark Streaming to control the receiving rate based on the
2015-09-08 17:54:43 -04:00
current batch scheduling delays and processing times so that the system receives
2015-09-21 14:46:39 -04:00
only as fast as the system can process. Internally, this dynamically sets the
2015-09-08 17:54:43 -04:00
maximum receiving rate of receivers. This rate is upper bounded by the values
2015-09-21 14:46:39 -04:00
< code > spark.streaming.receiver.maxRate< / code > and < code > spark.streaming.kafka.maxRatePerPartition< / code >
2015-09-08 17:54:43 -04:00
if they are set (see below).
< / td >
< / tr >
2016-02-16 22:43:17 -05:00
< tr >
< td > < code > spark.streaming.backpressure.initialRate< / code > < / td >
< td > not set< / td >
< td >
This is the initial maximum receiving rate at which each receiver will receive data for the
first batch when the backpressure mechanism is enabled.
< / td >
< / 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-09-27 16:26:54 -04:00
< tr >
< td > < code > spark.streaming.stopGracefullyOnShutdown< / code > < / td >
< td > false< / td >
< td >
2015-12-15 21:24:23 -05:00
If < code > true< / code > , Spark shuts down the < code > StreamingContext< / code > gracefully on JVM
2015-09-27 16:26:54 -04:00
shutdown rather than immediately.
< / 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 >
2016-01-07 20:37:46 -05:00
< tr >
< td > < code > spark.streaming.driver.writeAheadLog.closeFileAfterWrite< / code > < / td >
< td > false< / td >
< td >
Whether to close the file after writing a write ahead log record on the driver. Set this to 'true'
when you want to use S3 (or any file system that does not support flushing) for the metadata WAL
on the driver.
< / td >
< / tr >
< tr >
< td > < code > spark.streaming.receiver.writeAheadLog.closeFileAfterWrite< / code > < / td >
< td > false< / td >
< td >
Whether to close the file after writing a write ahead log record on the receivers. Set this to 'true'
when you want to use S3 (or any file system that does not support flushing) for the data WAL
on the receivers.
< / td >
< / tr >
2012-09-26 22:17:58 -04:00
< / table >
2012-09-03 02:05:40 -04:00
2017-01-07 14:15:51 -05:00
### SparkR
2015-06-10 16:18:48 -04:00
< table class = "table" >
< tr > < th > Property Name< / th > < th > Default< / th > < th > Meaning< / th > < / tr >
< tr >
< td > < code > spark.r.numRBackendThreads< / code > < / td >
< td > 2< / td >
< td >
Number of threads used by RBackend to handle RPC calls from SparkR package.
< / td >
< / tr >
2015-10-24 00:38:04 -04:00
< tr >
< td > < code > spark.r.command< / code > < / td >
< td > Rscript< / td >
< td >
Executable for executing R scripts in cluster modes for both driver and workers.
< / td >
< / tr >
< tr >
< td > < code > spark.r.driver.command< / code > < / td >
< td > spark.r.command< / td >
< td >
Executable for executing R scripts in client modes for driver. Ignored in cluster modes.
< / td >
< / tr >
2016-08-31 03:20:41 -04:00
< tr >
< td > < code > spark.r.shell.command< / code > < / td >
< td > R< / td >
< td >
Executable for executing sparkR shell in client modes for driver. Ignored in cluster modes. It is the same as environment variable < code > SPARKR_DRIVER_R< / code > , but take precedence over it.
< code > spark.r.shell.command< / code > is used for sparkR shell while < code > spark.r.driver.command< / code > is used for running R script.
< / td >
< / tr >
2016-10-30 19:17:23 -04:00
< tr >
< td > < code > spark.r.backendConnectionTimeout< / code > < / td >
< td > 6000< / td >
< td >
Connection timeout set by R process on its connection to RBackend in seconds.
< / td >
< / tr >
< tr >
< td > < code > spark.r.heartBeatInterval< / code > < / td >
< td > 100< / td >
< td >
2016-11-16 05:34:56 -05:00
Interval for heartbeats sent from SparkR backend to R process to prevent connection timeout.
2016-10-30 19:17:23 -04:00
< / td >
< / tr >
2015-06-10 16:18:48 -04:00
< / table >
2017-01-07 14:15:51 -05:00
### Deploy
2016-02-01 15:45:02 -05:00
< table class = "table" >
2016-02-21 18:27:07 -05:00
< tr > < th > Property Name< / th > < th > Default< / th > < th > Meaning< / th > < / tr >
2016-02-01 15:45:02 -05:00
< tr >
< td > < code > spark.deploy.recoveryMode< / code > < / td >
< td > NONE< / td >
< td > The recovery mode setting to recover submitted Spark jobs with cluster mode when it failed and relaunches.
This is only applicable for cluster mode when running with Standalone or Mesos.< / td >
< / tr >
< tr >
< td > < code > spark.deploy.zookeeper.url< / code > < / td >
< td > None< / td >
< td > When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this configuration is used to set the zookeeper URL to connect to.</ td >
< / tr >
< tr >
< td > < code > spark.deploy.zookeeper.dir< / code > < / td >
< td > None< / td >
< td > When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this configuration is used to set the zookeeper directory to store recovery state.</ td >
< / tr >
< / table >
2017-01-07 14:15:51 -05: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:
2017-01-07 14:15:51 -05:00
#### [YARN](running-on-yarn.html#configuration)
2015-03-14 13:28:13 -04:00
2017-01-07 14:15:51 -05:00
#### [Mesos](running-on-mesos.html#configuration)
2015-03-14 13:28:13 -04:00
2017-01-07 14:15:51 -05:00
#### [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 >
2015-09-21 14:46:39 -04:00
< td > Location where Java is installed (if it's not on your default < code > PATH< / code > ).< / td >
2014-05-21 21:49:12 -04:00
< / tr >
< tr >
< td > < code > PYSPARK_PYTHON< / code > < / td >
2016-08-11 23:08:25 -04:00
< td > Python binary executable to use for PySpark in both driver and workers (default is < code > python2.7< / code > if available, otherwise < code > python< / code > ).
Property < code > spark.pyspark.python< / code > take precedence if it is set< / td >
2015-08-19 01:11:27 -04:00
< / tr >
< tr >
< td > < code > PYSPARK_DRIVER_PYTHON< / code > < / td >
2016-08-11 23:08:25 -04:00
< td > Python binary executable to use for PySpark in driver only (default is < code > PYSPARK_PYTHON< / code > ).
Property < code > spark.pyspark.driver.python< / code > take precedence if it is set< / td >
2014-05-21 21:49:12 -04:00
< / tr >
2015-10-24 00:38:04 -04:00
< tr >
< td > < code > SPARKR_DRIVER_R< / code > < / td >
2016-08-31 03:20:41 -04:00
< td > R binary executable to use for SparkR shell (default is < code > R< / code > ).
Property < code > spark.r.shell.command< / code > take precedence if it is set< / td >
2015-10-24 00:38:04 -04:00
< / tr >
2014-05-21 21:49:12 -04:00
< 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
2016-01-27 04:31:44 -05:00
Note: When running Spark on YARN in `cluster` mode, environment variables need to be set using the `spark.yarn.appMasterEnv.[EnvironmentVariableName]` property in your `conf/spark-defaults.conf` file. Environment variables that are set in `spark-env.sh` will not be reflected in the YARN Application Master process in `cluster` mode. See the [YARN-related Spark Properties ](running-on-yarn.html#spark-properties ) for more information.
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",
2016-02-21 18:27:07 -05:00
you can set SPARK_CONF_DIR. Spark will use the configuration files (spark-defaults.conf, spark-env.sh, log4j.properties, etc)
2014-10-03 13:03:15 -04:00
from this directory.
2015-11-01 07:25:49 -05:00
# Inheriting Hadoop Cluster Configuration
If you plan to read and write from HDFS using Spark, there are two Hadoop configuration files that
should be included on Spark's classpath:
* `hdfs-site.xml` , which provides default behaviors for the HDFS client.
* `core-site.xml` , which sets the default filesystem name.
The location of these configuration files varies across CDH and HDP versions, but
a common location is inside of `/etc/hadoop/conf` . Some tools, such as Cloudera Manager, create
configurations on-the-fly, but offer a mechanisms to download copies of them.
To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh`
to a location containing the configuration files.