spark-instrumented-optimizer/docs/configuration.md

2991 lines
114 KiB
Markdown
Raw Normal View History

---
layout: global
displayTitle: Spark Configuration
title: Configuration
license: |
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
---
* This will become a table of contents (this text will be scraped).
{:toc}
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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
a [SparkConf](api/scala/org/apache/spark/SparkConf.html) object, or through Java
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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`.
# Spark Properties
[SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. YARN - SparkPi was updated to not take in master as an argument; we should update the docs to reflect that. - The default YARN build guide should be in maven, not sbt. - This PR also adds a paragraph on steps to debug a YARN application. Standalone - Emphasize spark-submit more. Right now it's one small paragraph preceding the legacy way of launching through `org.apache.spark.deploy.Client`. - The way we set configurations / environment variables according to the old docs is outdated. This needs to reflect changes introduced by the Spark configuration changes we made. In general, this PR also adds a little more documentation on the new spark-shell, spark-submit, spark-defaults.conf etc here and there. Author: Andrew Or <andrewor14@gmail.com> Closes #701 from andrewor14/yarn-docs and squashes the following commits: e2c2312 [Andrew Or] Merge in changes in #752 (SPARK-1814) 25cfe7b [Andrew Or] Merge in the warning from SPARK-1753 a8c39c5 [Andrew Or] Minor changes 336bbd9 [Andrew Or] Tabs -> spaces 4d9d8f7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 041017a [Andrew Or] Abstract Spark submit documentation to cluster-overview.html 3cc0649 [Andrew Or] Detail how to set configurations + remove legacy instructions 5b7140a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 85a51fc [Andrew Or] Update run-example, spark-shell, configuration etc. c10e8c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 381fe32 [Andrew Or] Update docs for standalone mode 757c184 [Andrew Or] Add a note about the requirements for the debugging trick f8ca990 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 924f04c [Andrew Or] Revert addition of --deploy-mode d5fe17b [Andrew Or] Update the YARN docs
2014-05-12 22:44:14 -04:00
Spark properties control most application settings and are configured separately for each
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
application. These properties can be set directly on a
[SparkConf](api/scala/org/apache/spark/SparkConf.html) passed to your
[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
`SparkContext`. `SparkConf` allows you to configure some of the common properties
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
(e.g. master URL and application name), as well as arbitrary key-value pairs through the
`set()` method. For example, we could initialize an application with two threads as follows:
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
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.
{% highlight scala %}
val conf = new SparkConf()
.setMaster("local[2]")
[SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. YARN - SparkPi was updated to not take in master as an argument; we should update the docs to reflect that. - The default YARN build guide should be in maven, not sbt. - This PR also adds a paragraph on steps to debug a YARN application. Standalone - Emphasize spark-submit more. Right now it's one small paragraph preceding the legacy way of launching through `org.apache.spark.deploy.Client`. - The way we set configurations / environment variables according to the old docs is outdated. This needs to reflect changes introduced by the Spark configuration changes we made. In general, this PR also adds a little more documentation on the new spark-shell, spark-submit, spark-defaults.conf etc here and there. Author: Andrew Or <andrewor14@gmail.com> Closes #701 from andrewor14/yarn-docs and squashes the following commits: e2c2312 [Andrew Or] Merge in changes in #752 (SPARK-1814) 25cfe7b [Andrew Or] Merge in the warning from SPARK-1753 a8c39c5 [Andrew Or] Minor changes 336bbd9 [Andrew Or] Tabs -> spaces 4d9d8f7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 041017a [Andrew Or] Abstract Spark submit documentation to cluster-overview.html 3cc0649 [Andrew Or] Detail how to set configurations + remove legacy instructions 5b7140a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 85a51fc [Andrew Or] Update run-example, spark-shell, configuration etc. c10e8c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 381fe32 [Andrew Or] Update docs for standalone mode 757c184 [Andrew Or] Add a note about the requirements for the debugging trick f8ca990 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 924f04c [Andrew Or] Revert addition of --deploy-mode d5fe17b [Andrew Or] Update the YARN docs
2014-05-12 22:44:14 -04:00
.setAppName("CountingSheep")
val sc = new SparkContext(conf)
{% endhighlight %}
Note that we can have more than 1 thread in local mode, and in cases like Spark Streaming, we may
actually require more than 1 thread to prevent any sort of starvation issues.
Properties that specify some time duration should be configured with a unit of time.
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
The following format is accepted:
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
25ms (milliseconds)
5s (seconds)
10m or 10min (minutes)
3h (hours)
5d (days)
1y (years)
Properties that specify a byte size should be configured with a unit of size.
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
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)
While numbers without units are generally interpreted as bytes, a few are interpreted as KiB or MiB.
See documentation of individual configuration properties. Specifying units is desirable where
possible.
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
## Dynamically Loading Spark Properties
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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:
[SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. YARN - SparkPi was updated to not take in master as an argument; we should update the docs to reflect that. - The default YARN build guide should be in maven, not sbt. - This PR also adds a paragraph on steps to debug a YARN application. Standalone - Emphasize spark-submit more. Right now it's one small paragraph preceding the legacy way of launching through `org.apache.spark.deploy.Client`. - The way we set configurations / environment variables according to the old docs is outdated. This needs to reflect changes introduced by the Spark configuration changes we made. In general, this PR also adds a little more documentation on the new spark-shell, spark-submit, spark-defaults.conf etc here and there. Author: Andrew Or <andrewor14@gmail.com> Closes #701 from andrewor14/yarn-docs and squashes the following commits: e2c2312 [Andrew Or] Merge in changes in #752 (SPARK-1814) 25cfe7b [Andrew Or] Merge in the warning from SPARK-1753 a8c39c5 [Andrew Or] Minor changes 336bbd9 [Andrew Or] Tabs -> spaces 4d9d8f7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 041017a [Andrew Or] Abstract Spark submit documentation to cluster-overview.html 3cc0649 [Andrew Or] Detail how to set configurations + remove legacy instructions 5b7140a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 85a51fc [Andrew Or] Update run-example, spark-shell, configuration etc. c10e8c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 381fe32 [Andrew Or] Update docs for standalone mode 757c184 [Andrew Or] Add a note about the requirements for the debugging trick f8ca990 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 924f04c [Andrew Or] Revert addition of --deploy-mode d5fe17b [Andrew Or] Update the YARN docs
2014-05-12 22:44:14 -04:00
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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 %}
./bin/spark-submit --name "My app" --master local[4] --conf spark.eventLog.enabled=false
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
--conf "spark.executor.extraJavaOptions=-XX:+PrintGCDetails -XX:+PrintGCTimeStamps" myApp.jar
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
{% endhighlight %}
The Spark shell and [`spark-submit`](submitting-applications.html)
tool support two ways to load configurations dynamically. The first is command line options,
such as `--master`, as shown above. `spark-submit` can accept any Spark property using the `--conf/-c`
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.
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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:
[SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. YARN - SparkPi was updated to not take in master as an argument; we should update the docs to reflect that. - The default YARN build guide should be in maven, not sbt. - This PR also adds a paragraph on steps to debug a YARN application. Standalone - Emphasize spark-submit more. Right now it's one small paragraph preceding the legacy way of launching through `org.apache.spark.deploy.Client`. - The way we set configurations / environment variables according to the old docs is outdated. This needs to reflect changes introduced by the Spark configuration changes we made. In general, this PR also adds a little more documentation on the new spark-shell, spark-submit, spark-defaults.conf etc here and there. Author: Andrew Or <andrewor14@gmail.com> Closes #701 from andrewor14/yarn-docs and squashes the following commits: e2c2312 [Andrew Or] Merge in changes in #752 (SPARK-1814) 25cfe7b [Andrew Or] Merge in the warning from SPARK-1753 a8c39c5 [Andrew Or] Minor changes 336bbd9 [Andrew Or] Tabs -> spaces 4d9d8f7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 041017a [Andrew Or] Abstract Spark submit documentation to cluster-overview.html 3cc0649 [Andrew Or] Detail how to set configurations + remove legacy instructions 5b7140a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 85a51fc [Andrew Or] Update run-example, spark-shell, configuration etc. c10e8c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 381fe32 [Andrew Or] Update docs for standalone mode 757c184 [Andrew Or] Add a note about the requirements for the debugging trick f8ca990 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 924f04c [Andrew Or] Revert addition of --deploy-mode d5fe17b [Andrew Or] Update the YARN docs
2014-05-12 22:44:14 -04:00
spark.master spark://5.6.7.8:7077
spark.executor.memory 4g
[SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. YARN - SparkPi was updated to not take in master as an argument; we should update the docs to reflect that. - The default YARN build guide should be in maven, not sbt. - This PR also adds a paragraph on steps to debug a YARN application. Standalone - Emphasize spark-submit more. Right now it's one small paragraph preceding the legacy way of launching through `org.apache.spark.deploy.Client`. - The way we set configurations / environment variables according to the old docs is outdated. This needs to reflect changes introduced by the Spark configuration changes we made. In general, this PR also adds a little more documentation on the new spark-shell, spark-submit, spark-defaults.conf etc here and there. Author: Andrew Or <andrewor14@gmail.com> Closes #701 from andrewor14/yarn-docs and squashes the following commits: e2c2312 [Andrew Or] Merge in changes in #752 (SPARK-1814) 25cfe7b [Andrew Or] Merge in the warning from SPARK-1753 a8c39c5 [Andrew Or] Minor changes 336bbd9 [Andrew Or] Tabs -> spaces 4d9d8f7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 041017a [Andrew Or] Abstract Spark submit documentation to cluster-overview.html 3cc0649 [Andrew Or] Detail how to set configurations + remove legacy instructions 5b7140a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 85a51fc [Andrew Or] Update run-example, spark-shell, configuration etc. c10e8c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 381fe32 [Andrew Or] Update docs for standalone mode 757c184 [Andrew Or] Add a note about the requirements for the debugging trick f8ca990 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 924f04c [Andrew Or] Revert addition of --deploy-mode d5fe17b [Andrew Or] Update the YARN docs
2014-05-12 22:44:14 -04:00
spark.eventLog.enabled true
spark.serializer org.apache.spark.serializer.KryoSerializer
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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
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.
[SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. YARN - SparkPi was updated to not take in master as an argument; we should update the docs to reflect that. - The default YARN build guide should be in maven, not sbt. - This PR also adds a paragraph on steps to debug a YARN application. Standalone - Emphasize spark-submit more. Right now it's one small paragraph preceding the legacy way of launching through `org.apache.spark.deploy.Client`. - The way we set configurations / environment variables according to the old docs is outdated. This needs to reflect changes introduced by the Spark configuration changes we made. In general, this PR also adds a little more documentation on the new spark-shell, spark-submit, spark-defaults.conf etc here and there. Author: Andrew Or <andrewor14@gmail.com> Closes #701 from andrewor14/yarn-docs and squashes the following commits: e2c2312 [Andrew Or] Merge in changes in #752 (SPARK-1814) 25cfe7b [Andrew Or] Merge in the warning from SPARK-1753 a8c39c5 [Andrew Or] Minor changes 336bbd9 [Andrew Or] Tabs -> spaces 4d9d8f7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 041017a [Andrew Or] Abstract Spark submit documentation to cluster-overview.html 3cc0649 [Andrew Or] Detail how to set configurations + remove legacy instructions 5b7140a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 85a51fc [Andrew Or] Update run-example, spark-shell, configuration etc. c10e8c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 381fe32 [Andrew Or] Update docs for standalone mode 757c184 [Andrew Or] Add a note about the requirements for the debugging trick f8ca990 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 924f04c [Andrew Or] Revert addition of --deploy-mode d5fe17b [Andrew Or] Update the YARN docs
2014-05-12 22:44:14 -04:00
Spark properties mainly can be divided into two kinds: one is related to deploy, like
"spark.driver.memory", "spark.executor.instances", this kind of properties may not be affected when
setting programmatically through `SparkConf` in runtime, or the behavior is depending on which
cluster manager and deploy mode you choose, so it would be suggested to set through configuration
file or `spark-submit` command line options; another is mainly related to Spark runtime control,
like "spark.task.maxFailures", this kind of properties can be set in either way.
## 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
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.
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
## Available Properties
[SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. YARN - SparkPi was updated to not take in master as an argument; we should update the docs to reflect that. - The default YARN build guide should be in maven, not sbt. - This PR also adds a paragraph on steps to debug a YARN application. Standalone - Emphasize spark-submit more. Right now it's one small paragraph preceding the legacy way of launching through `org.apache.spark.deploy.Client`. - The way we set configurations / environment variables according to the old docs is outdated. This needs to reflect changes introduced by the Spark configuration changes we made. In general, this PR also adds a little more documentation on the new spark-shell, spark-submit, spark-defaults.conf etc here and there. Author: Andrew Or <andrewor14@gmail.com> Closes #701 from andrewor14/yarn-docs and squashes the following commits: e2c2312 [Andrew Or] Merge in changes in #752 (SPARK-1814) 25cfe7b [Andrew Or] Merge in the warning from SPARK-1753 a8c39c5 [Andrew Or] Minor changes 336bbd9 [Andrew Or] Tabs -> spaces 4d9d8f7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 041017a [Andrew Or] Abstract Spark submit documentation to cluster-overview.html 3cc0649 [Andrew Or] Detail how to set configurations + remove legacy instructions 5b7140a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 85a51fc [Andrew Or] Update run-example, spark-shell, configuration etc. c10e8c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 381fe32 [Andrew Or] Update docs for standalone mode 757c184 [Andrew Or] Add a note about the requirements for the debugging trick f8ca990 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 924f04c [Andrew Or] Revert addition of --deploy-mode d5fe17b [Andrew Or] Update the YARN docs
2014-05-12 22:44:14 -04:00
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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:
### Application Properties
<table class="table">
2020-02-26 20:57:34 -05:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>0.9.0</td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</tr>
<tr>
<td><code>spark.driver.cores</code></td>
<td>1</td>
<td>
Number of cores to use for the driver process, only in cluster mode.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.3.0</td>
</tr>
<tr>
<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) in bytes. Should be at least 1M, or 0 for unlimited. Jobs will be aborted if the total
size is above this limit.
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>1.2.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.driver.memory</code></td>
<td>1g</td>
<td>
Amount of memory to use for the driver process, i.e. where SparkContext is initialized, in the
same format as JVM memory strings with a size unit suffix ("k", "m", "g" or "t")
(e.g. <code>512m</code>, <code>2g</code>).
<br />
<em>Note:</em> In client mode, this config must not be set through the <code>SparkConf</code>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
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.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.1.1</td>
</tr>
[SPARK-22646][K8S] Spark on Kubernetes - basic submission client This PR contains implementation of the basic submission client for the cluster mode of Spark on Kubernetes. It's step 2 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935). This addition is covered by the [SPIP](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) vote which passed on Aug 31. This PR and #19468 together form a MVP of Spark on Kubernetes that allows users to run Spark applications that use resources locally within the driver and executor containers on Kubernetes 1.6 and up. Some changes on pom and build/test setup are copied over from #19468 to make this PR self contained and testable. The submission client is mainly responsible for creating the Kubernetes pod that runs the Spark driver. It follows a step-based approach to construct the driver pod, as the code under the `submit.steps` package shows. The steps are orchestrated by `DriverConfigurationStepsOrchestrator`. `Client` creates the driver pod and waits for the application to complete if it's configured to do so, which is the case by default. This PR also contains Dockerfiles of the driver and executor images. They are included because some of the environment variables set in the code would not make sense without referring to the Dockerfiles. * The patch contains unit tests which are passing. * Manual testing: ./build/mvn -Pkubernetes clean package succeeded. * It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations. * There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI. * Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html cc rxin felixcheung mateiz (shepherd) k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926 Author: Yinan Li <liyinan926@gmail.com> Closes #19717 from liyinan926/spark-kubernetes-4.
2017-12-11 18:14:59 -05:00
<tr>
<td><code>spark.driver.memoryOverhead</code></td>
<td>driverMemory * 0.10, with minimum of 384 </td>
<td>
Amount of non-heap memory to be allocated per driver process in cluster mode, in MiB unless
otherwise specified. This is memory that accounts for things like VM overheads, interned strings,
other native overheads, etc. This tends to grow with the container size (typically 6-10%).
This option is currently supported on YARN, Mesos and Kubernetes.
<em>Note:</em> Non-heap memory includes off-heap memory
(when <code>spark.memory.offHeap.enabled=true</code>) and memory used by other driver processes
(e.g. python process that goes with a PySpark driver) and memory used by other non-driver
processes running in the same container. The maximum memory size of container to running
driver is determined by the sum of <code>spark.driver.memoryOverhead</code>
and <code>spark.driver.memory</code>.
[SPARK-22646][K8S] Spark on Kubernetes - basic submission client This PR contains implementation of the basic submission client for the cluster mode of Spark on Kubernetes. It's step 2 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935). This addition is covered by the [SPIP](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) vote which passed on Aug 31. This PR and #19468 together form a MVP of Spark on Kubernetes that allows users to run Spark applications that use resources locally within the driver and executor containers on Kubernetes 1.6 and up. Some changes on pom and build/test setup are copied over from #19468 to make this PR self contained and testable. The submission client is mainly responsible for creating the Kubernetes pod that runs the Spark driver. It follows a step-based approach to construct the driver pod, as the code under the `submit.steps` package shows. The steps are orchestrated by `DriverConfigurationStepsOrchestrator`. `Client` creates the driver pod and waits for the application to complete if it's configured to do so, which is the case by default. This PR also contains Dockerfiles of the driver and executor images. They are included because some of the environment variables set in the code would not make sense without referring to the Dockerfiles. * The patch contains unit tests which are passing. * Manual testing: ./build/mvn -Pkubernetes clean package succeeded. * It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations. * There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI. * Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html cc rxin felixcheung mateiz (shepherd) k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926 Author: Yinan Li <liyinan926@gmail.com> Closes #19717 from liyinan926/spark-kubernetes-4.
2017-12-11 18:14:59 -05:00
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>2.3.0</td>
[SPARK-22646][K8S] Spark on Kubernetes - basic submission client This PR contains implementation of the basic submission client for the cluster mode of Spark on Kubernetes. It's step 2 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935). This addition is covered by the [SPIP](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) vote which passed on Aug 31. This PR and #19468 together form a MVP of Spark on Kubernetes that allows users to run Spark applications that use resources locally within the driver and executor containers on Kubernetes 1.6 and up. Some changes on pom and build/test setup are copied over from #19468 to make this PR self contained and testable. The submission client is mainly responsible for creating the Kubernetes pod that runs the Spark driver. It follows a step-based approach to construct the driver pod, as the code under the `submit.steps` package shows. The steps are orchestrated by `DriverConfigurationStepsOrchestrator`. `Client` creates the driver pod and waits for the application to complete if it's configured to do so, which is the case by default. This PR also contains Dockerfiles of the driver and executor images. They are included because some of the environment variables set in the code would not make sense without referring to the Dockerfiles. * The patch contains unit tests which are passing. * Manual testing: ./build/mvn -Pkubernetes clean package succeeded. * It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations. * There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI. * Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html cc rxin felixcheung mateiz (shepherd) k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926 Author: Yinan Li <liyinan926@gmail.com> Closes #19717 from liyinan926/spark-kubernetes-4.
2017-12-11 18:14:59 -05:00
</tr>
[SPARK-27488][CORE] Driver interface to support GPU resources ## What changes were proposed in this pull request? Added the driver functionality to get the resources. The user interface is: SparkContext.resources - I called it this to match the TaskContext.resources api proposed in the other PR. Originally it was going to be called SparkContext.getResources but changed to be consistent, if people have strong feelings I can change it. There are 2 ways the driver can discover what resources it has. 1) user specifies a discoveryScript, this is similar to the executors and is meant for yarn and k8s where they don't tell you what you were allocated but you are running in isolated environment. 2) read the config spark.driver.resource.resourceName.addresses. The config is meant to be used with standalone mode where the Worker will have to assign what GPU addresses the Driver is allowed to use by setting that config. When the user runs a spark application, if they want the driver to have GPU's they would specify the conf spark.driver.resource.gpu.count=X where x is the number they want. If they are running on yarn or k8s they will also have to specify the discoveryScript as specified above, if they are on standalone mode and cluster is setup properly they wouldn't have to specify anything else. We could potentially get rid of the spark.driver.resources.gpu.addresses config which is really meant to be an internal config for worker to set if the standalone mode Worker wanted to write a discoveryScript out and set that for the user. I'll wait for the jira that implements that to decide if we can remove. - This PR also has changes to be consistent about using resourceName everywhere. - change the config names from POSTFIX to SUFFIX to be more consistent with other areas in Spark - Moved the config checks around a bit since now used by both executor and driver. Note those might overlap a bit with https://github.com/apache/spark/pull/24374 so we will have to figure out which one should go in first. ## How was this patch tested? Unit tests and manually test the interface. Closes #24615 from tgravescs/SPARK-27488. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-05-23 14:46:13 -04:00
<tr>
<td><code>spark.driver.resource.{resourceName}.amount</code></td>
[SPARK-27488][CORE] Driver interface to support GPU resources ## What changes were proposed in this pull request? Added the driver functionality to get the resources. The user interface is: SparkContext.resources - I called it this to match the TaskContext.resources api proposed in the other PR. Originally it was going to be called SparkContext.getResources but changed to be consistent, if people have strong feelings I can change it. There are 2 ways the driver can discover what resources it has. 1) user specifies a discoveryScript, this is similar to the executors and is meant for yarn and k8s where they don't tell you what you were allocated but you are running in isolated environment. 2) read the config spark.driver.resource.resourceName.addresses. The config is meant to be used with standalone mode where the Worker will have to assign what GPU addresses the Driver is allowed to use by setting that config. When the user runs a spark application, if they want the driver to have GPU's they would specify the conf spark.driver.resource.gpu.count=X where x is the number they want. If they are running on yarn or k8s they will also have to specify the discoveryScript as specified above, if they are on standalone mode and cluster is setup properly they wouldn't have to specify anything else. We could potentially get rid of the spark.driver.resources.gpu.addresses config which is really meant to be an internal config for worker to set if the standalone mode Worker wanted to write a discoveryScript out and set that for the user. I'll wait for the jira that implements that to decide if we can remove. - This PR also has changes to be consistent about using resourceName everywhere. - change the config names from POSTFIX to SUFFIX to be more consistent with other areas in Spark - Moved the config checks around a bit since now used by both executor and driver. Note those might overlap a bit with https://github.com/apache/spark/pull/24374 so we will have to figure out which one should go in first. ## How was this patch tested? Unit tests and manually test the interface. Closes #24615 from tgravescs/SPARK-27488. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-05-23 14:46:13 -04:00
<td>0</td>
<td>
Amount of a particular resource type to use on the driver.
[SPARK-27488][CORE] Driver interface to support GPU resources ## What changes were proposed in this pull request? Added the driver functionality to get the resources. The user interface is: SparkContext.resources - I called it this to match the TaskContext.resources api proposed in the other PR. Originally it was going to be called SparkContext.getResources but changed to be consistent, if people have strong feelings I can change it. There are 2 ways the driver can discover what resources it has. 1) user specifies a discoveryScript, this is similar to the executors and is meant for yarn and k8s where they don't tell you what you were allocated but you are running in isolated environment. 2) read the config spark.driver.resource.resourceName.addresses. The config is meant to be used with standalone mode where the Worker will have to assign what GPU addresses the Driver is allowed to use by setting that config. When the user runs a spark application, if they want the driver to have GPU's they would specify the conf spark.driver.resource.gpu.count=X where x is the number they want. If they are running on yarn or k8s they will also have to specify the discoveryScript as specified above, if they are on standalone mode and cluster is setup properly they wouldn't have to specify anything else. We could potentially get rid of the spark.driver.resources.gpu.addresses config which is really meant to be an internal config for worker to set if the standalone mode Worker wanted to write a discoveryScript out and set that for the user. I'll wait for the jira that implements that to decide if we can remove. - This PR also has changes to be consistent about using resourceName everywhere. - change the config names from POSTFIX to SUFFIX to be more consistent with other areas in Spark - Moved the config checks around a bit since now used by both executor and driver. Note those might overlap a bit with https://github.com/apache/spark/pull/24374 so we will have to figure out which one should go in first. ## How was this patch tested? Unit tests and manually test the interface. Closes #24615 from tgravescs/SPARK-27488. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-05-23 14:46:13 -04:00
If this is used, you must also specify the
<code>spark.driver.resource.{resourceName}.discoveryScript</code>
for the driver to find the resource on startup.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
[SPARK-27488][CORE] Driver interface to support GPU resources ## What changes were proposed in this pull request? Added the driver functionality to get the resources. The user interface is: SparkContext.resources - I called it this to match the TaskContext.resources api proposed in the other PR. Originally it was going to be called SparkContext.getResources but changed to be consistent, if people have strong feelings I can change it. There are 2 ways the driver can discover what resources it has. 1) user specifies a discoveryScript, this is similar to the executors and is meant for yarn and k8s where they don't tell you what you were allocated but you are running in isolated environment. 2) read the config spark.driver.resource.resourceName.addresses. The config is meant to be used with standalone mode where the Worker will have to assign what GPU addresses the Driver is allowed to use by setting that config. When the user runs a spark application, if they want the driver to have GPU's they would specify the conf spark.driver.resource.gpu.count=X where x is the number they want. If they are running on yarn or k8s they will also have to specify the discoveryScript as specified above, if they are on standalone mode and cluster is setup properly they wouldn't have to specify anything else. We could potentially get rid of the spark.driver.resources.gpu.addresses config which is really meant to be an internal config for worker to set if the standalone mode Worker wanted to write a discoveryScript out and set that for the user. I'll wait for the jira that implements that to decide if we can remove. - This PR also has changes to be consistent about using resourceName everywhere. - change the config names from POSTFIX to SUFFIX to be more consistent with other areas in Spark - Moved the config checks around a bit since now used by both executor and driver. Note those might overlap a bit with https://github.com/apache/spark/pull/24374 so we will have to figure out which one should go in first. ## How was this patch tested? Unit tests and manually test the interface. Closes #24615 from tgravescs/SPARK-27488. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-05-23 14:46:13 -04:00
</tr>
<tr>
<td><code>spark.driver.resource.{resourceName}.discoveryScript</code></td>
<td>None</td>
<td>
A script for the driver to run to discover a particular resource type. This should
write to STDOUT a JSON string in the format of the ResourceInformation class. This has a
name and an array of addresses. For a client-submitted driver, discovery script must assign
different resource addresses to this driver comparing to other drivers on the same host.
[SPARK-27488][CORE] Driver interface to support GPU resources ## What changes were proposed in this pull request? Added the driver functionality to get the resources. The user interface is: SparkContext.resources - I called it this to match the TaskContext.resources api proposed in the other PR. Originally it was going to be called SparkContext.getResources but changed to be consistent, if people have strong feelings I can change it. There are 2 ways the driver can discover what resources it has. 1) user specifies a discoveryScript, this is similar to the executors and is meant for yarn and k8s where they don't tell you what you were allocated but you are running in isolated environment. 2) read the config spark.driver.resource.resourceName.addresses. The config is meant to be used with standalone mode where the Worker will have to assign what GPU addresses the Driver is allowed to use by setting that config. When the user runs a spark application, if they want the driver to have GPU's they would specify the conf spark.driver.resource.gpu.count=X where x is the number they want. If they are running on yarn or k8s they will also have to specify the discoveryScript as specified above, if they are on standalone mode and cluster is setup properly they wouldn't have to specify anything else. We could potentially get rid of the spark.driver.resources.gpu.addresses config which is really meant to be an internal config for worker to set if the standalone mode Worker wanted to write a discoveryScript out and set that for the user. I'll wait for the jira that implements that to decide if we can remove. - This PR also has changes to be consistent about using resourceName everywhere. - change the config names from POSTFIX to SUFFIX to be more consistent with other areas in Spark - Moved the config checks around a bit since now used by both executor and driver. Note those might overlap a bit with https://github.com/apache/spark/pull/24374 so we will have to figure out which one should go in first. ## How was this patch tested? Unit tests and manually test the interface. Closes #24615 from tgravescs/SPARK-27488. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-05-23 14:46:13 -04:00
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
[SPARK-27488][CORE] Driver interface to support GPU resources ## What changes were proposed in this pull request? Added the driver functionality to get the resources. The user interface is: SparkContext.resources - I called it this to match the TaskContext.resources api proposed in the other PR. Originally it was going to be called SparkContext.getResources but changed to be consistent, if people have strong feelings I can change it. There are 2 ways the driver can discover what resources it has. 1) user specifies a discoveryScript, this is similar to the executors and is meant for yarn and k8s where they don't tell you what you were allocated but you are running in isolated environment. 2) read the config spark.driver.resource.resourceName.addresses. The config is meant to be used with standalone mode where the Worker will have to assign what GPU addresses the Driver is allowed to use by setting that config. When the user runs a spark application, if they want the driver to have GPU's they would specify the conf spark.driver.resource.gpu.count=X where x is the number they want. If they are running on yarn or k8s they will also have to specify the discoveryScript as specified above, if they are on standalone mode and cluster is setup properly they wouldn't have to specify anything else. We could potentially get rid of the spark.driver.resources.gpu.addresses config which is really meant to be an internal config for worker to set if the standalone mode Worker wanted to write a discoveryScript out and set that for the user. I'll wait for the jira that implements that to decide if we can remove. - This PR also has changes to be consistent about using resourceName everywhere. - change the config names from POSTFIX to SUFFIX to be more consistent with other areas in Spark - Moved the config checks around a bit since now used by both executor and driver. Note those might overlap a bit with https://github.com/apache/spark/pull/24374 so we will have to figure out which one should go in first. ## How was this patch tested? Unit tests and manually test the interface. Closes #24615 from tgravescs/SPARK-27488. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-05-23 14:46:13 -04:00
</tr>
<tr>
<td><code>spark.driver.resource.{resourceName}.vendor</code></td>
<td>None</td>
<td>
Vendor of the resources to use for the driver. This option is currently
only supported on Kubernetes and is actually both the vendor and domain following
the Kubernetes device plugin naming convention. (e.g. For GPUs on Kubernetes
this config would be set to nvidia.com or amd.com)
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
[SPARK-30689][CORE][YARN] Add resource discovery plugin api to support YARN versions with resource scheduling ### What changes were proposed in this pull request? This change is to allow custom resource scheduler (GPUs,FPGAs,etc) resource discovery to be more flexible. Users are asking for it to work with hadoop 2.x versions that do not support resource scheduling in YARN and/or also they may not run in an isolated environment. This change creates a plugin api that users can write their own resource discovery class that allows a lot more flexibility. The user can chain plugins for different resource types. The user specified plugins execute in the order specified and will fall back to use the discovery script plugin if they don't return information for a particular resource. I had to open up a few of the classes to be public and change them to not be case classes and make them developer api in order for the the plugin to get enough information it needs. I also relaxed the yarn side so that if yarn isn't configured for resource scheduling we just warn and go on. This helps users that have yarn 3.1 but haven't configured the resource scheduling side on their cluster yet, or aren't running in isolated environment. The user would configured this like: --conf spark.resources.discovery.plugin="org.apache.spark.resource.ResourceDiscoveryFPGAPlugin, org.apache.spark.resource.ResourceDiscoveryGPUPlugin" Note the executor side had to be wrapped with a classloader to make sure we include the user classpath for jars they specified on submission. Note this is more flexible because the discovery script has limitations such as spawning it in a separate process. This means if you are trying to allocate resources in that process they might be released when the script returns. Other things are the class makes it more flexible to be able to integrate with existing systems and solutions for assigning resources. ### Why are the changes needed? to more easily use spark resource scheduling with older versions of hadoop or in non-isolated enivronments. ### Does this PR introduce any user-facing change? Yes a plugin api ### How was this patch tested? Unit tests added and manual testing done on yarn and standalone modes. Closes #27410 from tgravescs/hadoop27spark3. Lead-authored-by: Thomas Graves <tgraves@nvidia.com> Co-authored-by: Thomas Graves <tgraves@apache.org> Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-01-31 23:20:28 -05:00
<tr>
<td><code>spark.resources.discoveryPlugin</code></td>
[SPARK-30689][CORE][YARN] Add resource discovery plugin api to support YARN versions with resource scheduling ### What changes were proposed in this pull request? This change is to allow custom resource scheduler (GPUs,FPGAs,etc) resource discovery to be more flexible. Users are asking for it to work with hadoop 2.x versions that do not support resource scheduling in YARN and/or also they may not run in an isolated environment. This change creates a plugin api that users can write their own resource discovery class that allows a lot more flexibility. The user can chain plugins for different resource types. The user specified plugins execute in the order specified and will fall back to use the discovery script plugin if they don't return information for a particular resource. I had to open up a few of the classes to be public and change them to not be case classes and make them developer api in order for the the plugin to get enough information it needs. I also relaxed the yarn side so that if yarn isn't configured for resource scheduling we just warn and go on. This helps users that have yarn 3.1 but haven't configured the resource scheduling side on their cluster yet, or aren't running in isolated environment. The user would configured this like: --conf spark.resources.discovery.plugin="org.apache.spark.resource.ResourceDiscoveryFPGAPlugin, org.apache.spark.resource.ResourceDiscoveryGPUPlugin" Note the executor side had to be wrapped with a classloader to make sure we include the user classpath for jars they specified on submission. Note this is more flexible because the discovery script has limitations such as spawning it in a separate process. This means if you are trying to allocate resources in that process they might be released when the script returns. Other things are the class makes it more flexible to be able to integrate with existing systems and solutions for assigning resources. ### Why are the changes needed? to more easily use spark resource scheduling with older versions of hadoop or in non-isolated enivronments. ### Does this PR introduce any user-facing change? Yes a plugin api ### How was this patch tested? Unit tests added and manual testing done on yarn and standalone modes. Closes #27410 from tgravescs/hadoop27spark3. Lead-authored-by: Thomas Graves <tgraves@nvidia.com> Co-authored-by: Thomas Graves <tgraves@apache.org> Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-01-31 23:20:28 -05:00
<td>org.apache.spark.resource.ResourceDiscoveryScriptPlugin</td>
<td>
Comma-separated list of class names implementing
org.apache.spark.api.resource.ResourceDiscoveryPlugin to load into the application.
This is for advanced users to replace the resource discovery class with a
custom implementation. Spark will try each class specified until one of them
returns the resource information for that resource. It tries the discovery
script last if none of the plugins return information for that resource.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
[SPARK-30689][CORE][YARN] Add resource discovery plugin api to support YARN versions with resource scheduling ### What changes were proposed in this pull request? This change is to allow custom resource scheduler (GPUs,FPGAs,etc) resource discovery to be more flexible. Users are asking for it to work with hadoop 2.x versions that do not support resource scheduling in YARN and/or also they may not run in an isolated environment. This change creates a plugin api that users can write their own resource discovery class that allows a lot more flexibility. The user can chain plugins for different resource types. The user specified plugins execute in the order specified and will fall back to use the discovery script plugin if they don't return information for a particular resource. I had to open up a few of the classes to be public and change them to not be case classes and make them developer api in order for the the plugin to get enough information it needs. I also relaxed the yarn side so that if yarn isn't configured for resource scheduling we just warn and go on. This helps users that have yarn 3.1 but haven't configured the resource scheduling side on their cluster yet, or aren't running in isolated environment. The user would configured this like: --conf spark.resources.discovery.plugin="org.apache.spark.resource.ResourceDiscoveryFPGAPlugin, org.apache.spark.resource.ResourceDiscoveryGPUPlugin" Note the executor side had to be wrapped with a classloader to make sure we include the user classpath for jars they specified on submission. Note this is more flexible because the discovery script has limitations such as spawning it in a separate process. This means if you are trying to allocate resources in that process they might be released when the script returns. Other things are the class makes it more flexible to be able to integrate with existing systems and solutions for assigning resources. ### Why are the changes needed? to more easily use spark resource scheduling with older versions of hadoop or in non-isolated enivronments. ### Does this PR introduce any user-facing change? Yes a plugin api ### How was this patch tested? Unit tests added and manual testing done on yarn and standalone modes. Closes #27410 from tgravescs/hadoop27spark3. Lead-authored-by: Thomas Graves <tgraves@nvidia.com> Co-authored-by: Thomas Graves <tgraves@apache.org> Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-01-31 23:20:28 -05:00
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.executor.memory</code></td>
<td>1g</td>
<td>
Amount of memory to use per executor process, in the same format as JVM memory strings with
a size unit suffix ("k", "m", "g" or "t") (e.g. <code>512m</code>, <code>2g</code>).
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>0.7.0</td>
</tr>
2018-08-28 15:31:33 -04:00
<tr>
<td><code>spark.executor.pyspark.memory</code></td>
<td>Not set</td>
<td>
The amount of memory to be allocated to PySpark in each executor, in MiB
unless otherwise specified. If set, PySpark memory for an executor will be
limited to this amount. If not set, Spark will not limit Python's memory use
and it is up to the application to avoid exceeding the overhead memory space
shared with other non-JVM processes. When PySpark is run in YARN or Kubernetes, this memory
2018-08-28 15:31:33 -04:00
is added to executor resource requests.
<br/>
<em>Note:</em> This feature is dependent on Python's `resource` module; therefore, the behaviors and
limitations are inherited. For instance, Windows does not support resource limiting and actual
resource is not limited on MacOS.
2018-08-28 15:31:33 -04:00
</td>
2020-02-26 20:57:34 -05:00
<td>2.4.0</td>
2018-08-28 15:31:33 -04:00
</tr>
[SPARK-22646][K8S] Spark on Kubernetes - basic submission client This PR contains implementation of the basic submission client for the cluster mode of Spark on Kubernetes. It's step 2 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935). This addition is covered by the [SPIP](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) vote which passed on Aug 31. This PR and #19468 together form a MVP of Spark on Kubernetes that allows users to run Spark applications that use resources locally within the driver and executor containers on Kubernetes 1.6 and up. Some changes on pom and build/test setup are copied over from #19468 to make this PR self contained and testable. The submission client is mainly responsible for creating the Kubernetes pod that runs the Spark driver. It follows a step-based approach to construct the driver pod, as the code under the `submit.steps` package shows. The steps are orchestrated by `DriverConfigurationStepsOrchestrator`. `Client` creates the driver pod and waits for the application to complete if it's configured to do so, which is the case by default. This PR also contains Dockerfiles of the driver and executor images. They are included because some of the environment variables set in the code would not make sense without referring to the Dockerfiles. * The patch contains unit tests which are passing. * Manual testing: ./build/mvn -Pkubernetes clean package succeeded. * It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations. * There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI. * Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html cc rxin felixcheung mateiz (shepherd) k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926 Author: Yinan Li <liyinan926@gmail.com> Closes #19717 from liyinan926/spark-kubernetes-4.
2017-12-11 18:14:59 -05:00
<tr>
<td><code>spark.executor.memoryOverhead</code></td>
<td>executorMemory * 0.10, with minimum of 384 </td>
<td>
Amount of additional memory to be allocated per executor process in cluster mode, in MiB unless
otherwise specified. This is memory that accounts for things like VM overheads, interned strings,
other native overheads, etc. This tends to grow with the executor size (typically 6-10%).
This option is currently supported on YARN and Kubernetes.
<br/>
<em>Note:</em> Additional memory includes PySpark executor memory
(when <code>spark.executor.pyspark.memory</code> is not configured) and memory used by other
non-executor processes running in the same container. The maximum memory size of container to
running executor is determined by the sum of <code>spark.executor.memoryOverhead</code>,
<code>spark.executor.memory</code>, <code>spark.memory.offHeap.size</code> and
<code>spark.executor.pyspark.memory</code>.
[SPARK-22646][K8S] Spark on Kubernetes - basic submission client This PR contains implementation of the basic submission client for the cluster mode of Spark on Kubernetes. It's step 2 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935). This addition is covered by the [SPIP](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) vote which passed on Aug 31. This PR and #19468 together form a MVP of Spark on Kubernetes that allows users to run Spark applications that use resources locally within the driver and executor containers on Kubernetes 1.6 and up. Some changes on pom and build/test setup are copied over from #19468 to make this PR self contained and testable. The submission client is mainly responsible for creating the Kubernetes pod that runs the Spark driver. It follows a step-based approach to construct the driver pod, as the code under the `submit.steps` package shows. The steps are orchestrated by `DriverConfigurationStepsOrchestrator`. `Client` creates the driver pod and waits for the application to complete if it's configured to do so, which is the case by default. This PR also contains Dockerfiles of the driver and executor images. They are included because some of the environment variables set in the code would not make sense without referring to the Dockerfiles. * The patch contains unit tests which are passing. * Manual testing: ./build/mvn -Pkubernetes clean package succeeded. * It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations. * There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI. * Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html cc rxin felixcheung mateiz (shepherd) k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926 Author: Yinan Li <liyinan926@gmail.com> Closes #19717 from liyinan926/spark-kubernetes-4.
2017-12-11 18:14:59 -05:00
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>2.3.0</td>
[SPARK-22646][K8S] Spark on Kubernetes - basic submission client This PR contains implementation of the basic submission client for the cluster mode of Spark on Kubernetes. It's step 2 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935). This addition is covered by the [SPIP](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) vote which passed on Aug 31. This PR and #19468 together form a MVP of Spark on Kubernetes that allows users to run Spark applications that use resources locally within the driver and executor containers on Kubernetes 1.6 and up. Some changes on pom and build/test setup are copied over from #19468 to make this PR self contained and testable. The submission client is mainly responsible for creating the Kubernetes pod that runs the Spark driver. It follows a step-based approach to construct the driver pod, as the code under the `submit.steps` package shows. The steps are orchestrated by `DriverConfigurationStepsOrchestrator`. `Client` creates the driver pod and waits for the application to complete if it's configured to do so, which is the case by default. This PR also contains Dockerfiles of the driver and executor images. They are included because some of the environment variables set in the code would not make sense without referring to the Dockerfiles. * The patch contains unit tests which are passing. * Manual testing: ./build/mvn -Pkubernetes clean package succeeded. * It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations. * There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI. * Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html cc rxin felixcheung mateiz (shepherd) k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926 Author: Yinan Li <liyinan926@gmail.com> Closes #19717 from liyinan926/spark-kubernetes-4.
2017-12-11 18:14:59 -05:00
</tr>
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
<tr>
<td><code>spark.executor.resource.{resourceName}.amount</code></td>
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
<td>0</td>
<td>
Amount of a particular resource type to use per executor process.
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
If this is used, you must also specify the
[SPARK-27488][CORE] Driver interface to support GPU resources ## What changes were proposed in this pull request? Added the driver functionality to get the resources. The user interface is: SparkContext.resources - I called it this to match the TaskContext.resources api proposed in the other PR. Originally it was going to be called SparkContext.getResources but changed to be consistent, if people have strong feelings I can change it. There are 2 ways the driver can discover what resources it has. 1) user specifies a discoveryScript, this is similar to the executors and is meant for yarn and k8s where they don't tell you what you were allocated but you are running in isolated environment. 2) read the config spark.driver.resource.resourceName.addresses. The config is meant to be used with standalone mode where the Worker will have to assign what GPU addresses the Driver is allowed to use by setting that config. When the user runs a spark application, if they want the driver to have GPU's they would specify the conf spark.driver.resource.gpu.count=X where x is the number they want. If they are running on yarn or k8s they will also have to specify the discoveryScript as specified above, if they are on standalone mode and cluster is setup properly they wouldn't have to specify anything else. We could potentially get rid of the spark.driver.resources.gpu.addresses config which is really meant to be an internal config for worker to set if the standalone mode Worker wanted to write a discoveryScript out and set that for the user. I'll wait for the jira that implements that to decide if we can remove. - This PR also has changes to be consistent about using resourceName everywhere. - change the config names from POSTFIX to SUFFIX to be more consistent with other areas in Spark - Moved the config checks around a bit since now used by both executor and driver. Note those might overlap a bit with https://github.com/apache/spark/pull/24374 so we will have to figure out which one should go in first. ## How was this patch tested? Unit tests and manually test the interface. Closes #24615 from tgravescs/SPARK-27488. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-05-23 14:46:13 -04:00
<code>spark.executor.resource.{resourceName}.discoveryScript</code>
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
for the executor to find the resource on startup.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
</tr>
<tr>
[SPARK-27488][CORE] Driver interface to support GPU resources ## What changes were proposed in this pull request? Added the driver functionality to get the resources. The user interface is: SparkContext.resources - I called it this to match the TaskContext.resources api proposed in the other PR. Originally it was going to be called SparkContext.getResources but changed to be consistent, if people have strong feelings I can change it. There are 2 ways the driver can discover what resources it has. 1) user specifies a discoveryScript, this is similar to the executors and is meant for yarn and k8s where they don't tell you what you were allocated but you are running in isolated environment. 2) read the config spark.driver.resource.resourceName.addresses. The config is meant to be used with standalone mode where the Worker will have to assign what GPU addresses the Driver is allowed to use by setting that config. When the user runs a spark application, if they want the driver to have GPU's they would specify the conf spark.driver.resource.gpu.count=X where x is the number they want. If they are running on yarn or k8s they will also have to specify the discoveryScript as specified above, if they are on standalone mode and cluster is setup properly they wouldn't have to specify anything else. We could potentially get rid of the spark.driver.resources.gpu.addresses config which is really meant to be an internal config for worker to set if the standalone mode Worker wanted to write a discoveryScript out and set that for the user. I'll wait for the jira that implements that to decide if we can remove. - This PR also has changes to be consistent about using resourceName everywhere. - change the config names from POSTFIX to SUFFIX to be more consistent with other areas in Spark - Moved the config checks around a bit since now used by both executor and driver. Note those might overlap a bit with https://github.com/apache/spark/pull/24374 so we will have to figure out which one should go in first. ## How was this patch tested? Unit tests and manually test the interface. Closes #24615 from tgravescs/SPARK-27488. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-05-23 14:46:13 -04:00
<td><code>spark.executor.resource.{resourceName}.discoveryScript</code></td>
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
<td>None</td>
<td>
A script for the executor to run to discover a particular resource type. This should
write to STDOUT a JSON string in the format of the ResourceInformation class. This has a
name and an array of addresses.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
</tr>
<tr>
<td><code>spark.executor.resource.{resourceName}.vendor</code></td>
<td>None</td>
<td>
Vendor of the resources to use for the executors. This option is currently
only supported on Kubernetes and is actually both the vendor and domain following
the Kubernetes device plugin naming convention. (e.g. For GPUs on Kubernetes
this config would be set to nvidia.com or amd.com)
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.extraListeners</code></td>
<td>(none)</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>1.3.0</td>
</tr>
<tr>
<td><code>spark.local.dir</code></td>
<td>/tmp</td>
<td>
Directory to use for "scratch" space in Spark, including map output files and RDDs that get
stored on disk. This should be on a fast, local disk in your system. It can also be a
comma-separated list of multiple directories on different disks.
Clean up and simplify Spark configuration Over time as we've added more deployment modes, this have gotten a bit unwieldy with user-facing configuration options in Spark. Going forward we'll advise all users to run `spark-submit` to launch applications. This is a WIP patch but it makes the following improvements: 1. Improved `spark-env.sh.template` which was missing a lot of things users now set in that file. 2. Removes the shipping of SPARK_CLASSPATH, SPARK_JAVA_OPTS, and SPARK_LIBRARY_PATH to the executors on the cluster. This was an ugly hack. Instead it introduces config variables spark.executor.extraJavaOpts, spark.executor.extraLibraryPath, and spark.executor.extraClassPath. 3. Adds ability to set these same variables for the driver using `spark-submit`. 4. Allows you to load system properties from a `spark-defaults.conf` file when running `spark-submit`. This will allow setting both SparkConf options and other system properties utilized by `spark-submit`. 5. Made `SPARK_LOCAL_IP` an environment variable rather than a SparkConf property. This is more consistent with it being set on each node. Author: Patrick Wendell <pwendell@gmail.com> Closes #299 from pwendell/config-cleanup and squashes the following commits: 127f301 [Patrick Wendell] Improvements to testing a006464 [Patrick Wendell] Moving properties file template. b4b496c [Patrick Wendell] spark-defaults.properties -> spark-defaults.conf 0086939 [Patrick Wendell] Minor style fixes af09e3e [Patrick Wendell] Mention config file in docs and clean-up docs b16e6a2 [Patrick Wendell] Cleanup of spark-submit script and Scala quick start guide af0adf7 [Patrick Wendell] Automatically add user jar a56b125 [Patrick Wendell] Responses to Tom's review d50c388 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a762901 [Patrick Wendell] Fixing test failures ffa00fe [Patrick Wendell] Review feedback fda0301 [Patrick Wendell] Note 308f1f6 [Patrick Wendell] Properly escape quotes and other clean-up for YARN e83cd8f [Patrick Wendell] Changes to allow re-use of test applications be42f35 [Patrick Wendell] Handle case where SPARK_HOME is not set c2a2909 [Patrick Wendell] Test compile fixes 4ee6f9d [Patrick Wendell] Making YARN doc changes consistent afc9ed8 [Patrick Wendell] Cleaning up line limits and two compile errors. b08893b [Patrick Wendell] Additional improvements. ace4ead [Patrick Wendell] Responses to review feedback. b72d183 [Patrick Wendell] Review feedback for spark env file 46555c1 [Patrick Wendell] Review feedback and import clean-ups 437aed1 [Patrick Wendell] Small fix 761ebcd [Patrick Wendell] Library path and classpath for drivers 7cc70e4 [Patrick Wendell] Clean up terminology inside of spark-env script 5b0ba8e [Patrick Wendell] Don't ship executor envs 84cc5e5 [Patrick Wendell] Small clean-up 1f75238 [Patrick Wendell] SPARK_JAVA_OPTS --> SPARK_MASTER_OPTS for master settings 4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH 6eaf7d0 [Patrick Wendell] executorJavaOpts 0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS
2014-04-21 13:26:33 -04:00
<br/>
<em>Note:</em> This will be overridden by SPARK_LOCAL_DIRS (Standalone), MESOS_SANDBOX (Mesos) or
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
LOCAL_DIRS (YARN) environment variables set by the cluster manager.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>0.5.0</td>
</tr>
<tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td><code>spark.logConf</code></td>
<td>false</td>
<td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
Logs the effective SparkConf as INFO when a SparkContext is started.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>0.9.0</td>
</tr>
[SPARK-5411] Allow SparkListeners to be specified in SparkConf and loaded when creating SparkContext This patch introduces a new configuration option, `spark.extraListeners`, that allows SparkListeners to be specified in SparkConf and registered before the SparkContext is initialized. From the configuration documentation: > A comma-separated list of classes that implement SparkListener; 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. This motivation for this patch is to allow monitoring code to be easily injected into existing Spark programs without having to modify those programs' code. Author: Josh Rosen <joshrosen@databricks.com> Closes #4111 from JoshRosen/SPARK-5190-register-sparklistener-in-sc-constructor and squashes the following commits: 8370839 [Josh Rosen] Two minor fixes after merging with master 6e0122c [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-5190-register-sparklistener-in-sc-constructor 1a5b9a0 [Josh Rosen] Remove SPARK_EXTRA_LISTENERS environment variable. 2daff9b [Josh Rosen] Add a couple of explanatory comments for SPARK_EXTRA_LISTENERS. b9973da [Josh Rosen] Add test to ensure that conf and env var settings are merged, not overriden. d6f3113 [Josh Rosen] Use getConstructors() instead of try-catch to find right constructor. d0d276d [Josh Rosen] Move code into setupAndStartListenerBus() method b22b379 [Josh Rosen] Instantiate SparkListeners from classes listed in configurations. 9c0d8f1 [Josh Rosen] Revert "[SPARK-5190] Allow SparkListeners to be registered before SparkContext starts." 217ecc0 [Josh Rosen] Revert "Add addSparkListener to JavaSparkContext" 25988f3 [Josh Rosen] Add addSparkListener to JavaSparkContext 163ba19 [Josh Rosen] [SPARK-5190] Allow SparkListeners to be registered before SparkContext starts.
2015-02-04 20:18:03 -05:00
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.master</code></td>
[SPARK-5411] Allow SparkListeners to be specified in SparkConf and loaded when creating SparkContext This patch introduces a new configuration option, `spark.extraListeners`, that allows SparkListeners to be specified in SparkConf and registered before the SparkContext is initialized. From the configuration documentation: > A comma-separated list of classes that implement SparkListener; 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. This motivation for this patch is to allow monitoring code to be easily injected into existing Spark programs without having to modify those programs' code. Author: Josh Rosen <joshrosen@databricks.com> Closes #4111 from JoshRosen/SPARK-5190-register-sparklistener-in-sc-constructor and squashes the following commits: 8370839 [Josh Rosen] Two minor fixes after merging with master 6e0122c [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-5190-register-sparklistener-in-sc-constructor 1a5b9a0 [Josh Rosen] Remove SPARK_EXTRA_LISTENERS environment variable. 2daff9b [Josh Rosen] Add a couple of explanatory comments for SPARK_EXTRA_LISTENERS. b9973da [Josh Rosen] Add test to ensure that conf and env var settings are merged, not overriden. d6f3113 [Josh Rosen] Use getConstructors() instead of try-catch to find right constructor. d0d276d [Josh Rosen] Move code into setupAndStartListenerBus() method b22b379 [Josh Rosen] Instantiate SparkListeners from classes listed in configurations. 9c0d8f1 [Josh Rosen] Revert "[SPARK-5190] Allow SparkListeners to be registered before SparkContext starts." 217ecc0 [Josh Rosen] Revert "Add addSparkListener to JavaSparkContext" 25988f3 [Josh Rosen] Add addSparkListener to JavaSparkContext 163ba19 [Josh Rosen] [SPARK-5190] Allow SparkListeners to be registered before SparkContext starts.
2015-02-04 20:18:03 -05:00
<td>(none)</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>.
[SPARK-5411] Allow SparkListeners to be specified in SparkConf and loaded when creating SparkContext This patch introduces a new configuration option, `spark.extraListeners`, that allows SparkListeners to be specified in SparkConf and registered before the SparkContext is initialized. From the configuration documentation: > A comma-separated list of classes that implement SparkListener; 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. This motivation for this patch is to allow monitoring code to be easily injected into existing Spark programs without having to modify those programs' code. Author: Josh Rosen <joshrosen@databricks.com> Closes #4111 from JoshRosen/SPARK-5190-register-sparklistener-in-sc-constructor and squashes the following commits: 8370839 [Josh Rosen] Two minor fixes after merging with master 6e0122c [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-5190-register-sparklistener-in-sc-constructor 1a5b9a0 [Josh Rosen] Remove SPARK_EXTRA_LISTENERS environment variable. 2daff9b [Josh Rosen] Add a couple of explanatory comments for SPARK_EXTRA_LISTENERS. b9973da [Josh Rosen] Add test to ensure that conf and env var settings are merged, not overriden. d6f3113 [Josh Rosen] Use getConstructors() instead of try-catch to find right constructor. d0d276d [Josh Rosen] Move code into setupAndStartListenerBus() method b22b379 [Josh Rosen] Instantiate SparkListeners from classes listed in configurations. 9c0d8f1 [Josh Rosen] Revert "[SPARK-5190] Allow SparkListeners to be registered before SparkContext starts." 217ecc0 [Josh Rosen] Revert "Add addSparkListener to JavaSparkContext" 25988f3 [Josh Rosen] Add addSparkListener to JavaSparkContext 163ba19 [Josh Rosen] [SPARK-5190] Allow SparkListeners to be registered before SparkContext starts.
2015-02-04 20:18:03 -05:00
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>0.9.0</td>
[SPARK-5411] Allow SparkListeners to be specified in SparkConf and loaded when creating SparkContext This patch introduces a new configuration option, `spark.extraListeners`, that allows SparkListeners to be specified in SparkConf and registered before the SparkContext is initialized. From the configuration documentation: > A comma-separated list of classes that implement SparkListener; 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. This motivation for this patch is to allow monitoring code to be easily injected into existing Spark programs without having to modify those programs' code. Author: Josh Rosen <joshrosen@databricks.com> Closes #4111 from JoshRosen/SPARK-5190-register-sparklistener-in-sc-constructor and squashes the following commits: 8370839 [Josh Rosen] Two minor fixes after merging with master 6e0122c [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-5190-register-sparklistener-in-sc-constructor 1a5b9a0 [Josh Rosen] Remove SPARK_EXTRA_LISTENERS environment variable. 2daff9b [Josh Rosen] Add a couple of explanatory comments for SPARK_EXTRA_LISTENERS. b9973da [Josh Rosen] Add test to ensure that conf and env var settings are merged, not overriden. d6f3113 [Josh Rosen] Use getConstructors() instead of try-catch to find right constructor. d0d276d [Josh Rosen] Move code into setupAndStartListenerBus() method b22b379 [Josh Rosen] Instantiate SparkListeners from classes listed in configurations. 9c0d8f1 [Josh Rosen] Revert "[SPARK-5190] Allow SparkListeners to be registered before SparkContext starts." 217ecc0 [Josh Rosen] Revert "Add addSparkListener to JavaSparkContext" 25988f3 [Josh Rosen] Add addSparkListener to JavaSparkContext 163ba19 [Josh Rosen] [SPARK-5190] Allow SparkListeners to be registered before SparkContext starts.
2015-02-04 20:18:03 -05:00
</tr>
<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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.5.0</td>
</tr>
[SPARK-16759][CORE] Add a configuration property to pass caller contexts of upstream applications into Spark ## What changes were proposed in this pull request? Many applications take Spark as a computing engine and run on it. This PR adds a configuration property `spark.log.callerContext` that can be used by Spark's upstream applications (e.g. Oozie) to set up their caller contexts into Spark. In the end, Spark will combine its own caller context with the caller contexts of its upstream applications, and write them into Yarn RM log and HDFS audit log. The audit log has a config to truncate the caller contexts passed in (default 128). The caller contexts will be sent over rpc, so it should be concise. The call context written into HDFS log and Yarn log consists of two parts: the information `A` specified by Spark itself and the value `B` of `spark.log.callerContext` property. Currently `A` typically takes 64 to 74 characters, so `B` can have up to 50 characters (mentioned in the doc `running-on-yarn.md`) ## How was this patch tested? Manual tests. I have run some Spark applications with `spark.log.callerContext` configuration in Yarn client/cluster mode, and verified that the caller contexts were written into Yarn RM log and HDFS audit log correctly. The ways to configure `spark.log.callerContext` property: - In spark-defaults.conf: ``` spark.log.callerContext infoSpecifiedByUpstreamApp ``` - In app's source code: ``` val spark = SparkSession .builder .appName("SparkKMeans") .config("spark.log.callerContext", "infoSpecifiedByUpstreamApp") .getOrCreate() ``` When running on Spark Yarn cluster mode, the driver is unable to pass 'spark.log.callerContext' to Yarn client and AM since Yarn client and AM have already started before the driver performs `.config("spark.log.callerContext", "infoSpecifiedByUpstreamApp")`. The following example shows the command line used to submit a SparkKMeans application and the corresponding records in Yarn RM log and HDFS audit log. Command: ``` ./bin/spark-submit --verbose --executor-cores 3 --num-executors 1 --master yarn --deploy-mode client --class org.apache.spark.examples.SparkKMeans examples/target/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar hdfs://localhost:9000/lr_big.txt 2 5 ``` Yarn RM log: <img width="1440" alt="screen shot 2016-10-19 at 9 12 03 pm" src="https://cloud.githubusercontent.com/assets/8546874/19547050/7d2f278c-9649-11e6-9df8-8d5ff12609f0.png"> HDFS audit log: <img width="1400" alt="screen shot 2016-10-19 at 10 18 14 pm" src="https://cloud.githubusercontent.com/assets/8546874/19547102/096060ae-964a-11e6-981a-cb28efd5a058.png"> Author: Weiqing Yang <yangweiqing001@gmail.com> Closes #15563 from weiqingy/SPARK-16759.
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.2.0</td>
[SPARK-16759][CORE] Add a configuration property to pass caller contexts of upstream applications into Spark ## What changes were proposed in this pull request? Many applications take Spark as a computing engine and run on it. This PR adds a configuration property `spark.log.callerContext` that can be used by Spark's upstream applications (e.g. Oozie) to set up their caller contexts into Spark. In the end, Spark will combine its own caller context with the caller contexts of its upstream applications, and write them into Yarn RM log and HDFS audit log. The audit log has a config to truncate the caller contexts passed in (default 128). The caller contexts will be sent over rpc, so it should be concise. The call context written into HDFS log and Yarn log consists of two parts: the information `A` specified by Spark itself and the value `B` of `spark.log.callerContext` property. Currently `A` typically takes 64 to 74 characters, so `B` can have up to 50 characters (mentioned in the doc `running-on-yarn.md`) ## How was this patch tested? Manual tests. I have run some Spark applications with `spark.log.callerContext` configuration in Yarn client/cluster mode, and verified that the caller contexts were written into Yarn RM log and HDFS audit log correctly. The ways to configure `spark.log.callerContext` property: - In spark-defaults.conf: ``` spark.log.callerContext infoSpecifiedByUpstreamApp ``` - In app's source code: ``` val spark = SparkSession .builder .appName("SparkKMeans") .config("spark.log.callerContext", "infoSpecifiedByUpstreamApp") .getOrCreate() ``` When running on Spark Yarn cluster mode, the driver is unable to pass 'spark.log.callerContext' to Yarn client and AM since Yarn client and AM have already started before the driver performs `.config("spark.log.callerContext", "infoSpecifiedByUpstreamApp")`. The following example shows the command line used to submit a SparkKMeans application and the corresponding records in Yarn RM log and HDFS audit log. Command: ``` ./bin/spark-submit --verbose --executor-cores 3 --num-executors 1 --master yarn --deploy-mode client --class org.apache.spark.examples.SparkKMeans examples/target/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar hdfs://localhost:9000/lr_big.txt 2 5 ``` Yarn RM log: <img width="1440" alt="screen shot 2016-10-19 at 9 12 03 pm" src="https://cloud.githubusercontent.com/assets/8546874/19547050/7d2f278c-9649-11e6-9df8-8d5ff12609f0.png"> HDFS audit log: <img width="1400" alt="screen shot 2016-10-19 at 10 18 14 pm" src="https://cloud.githubusercontent.com/assets/8546874/19547102/096060ae-964a-11e6-981a-cb28efd5a058.png"> Author: Weiqing Yang <yangweiqing001@gmail.com> Closes #15563 from weiqingy/SPARK-16759.
2016-11-11 21:36:23 -05:00
</tr>
<tr>
<td><code>spark.driver.supervise</code></td>
<td>false</td>
<td>
If true, restarts the driver automatically if it fails with a non-zero exit status.
Only has effect in Spark standalone mode or Mesos cluster deploy mode.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>1.3.0</td>
</tr>
[SPARK-25118][CORE] Persist Driver Logs in Client mode to Hdfs Currently, we do not have a mechanism to collect driver logs if a user chooses to run their application in client mode. This is a big issue as admin teams need to create their own mechanisms to capture driver logs. This commit adds a logger which, if enabled, adds a local log appender to the root logger and asynchronously syncs it an application specific log file on hdfs (Spark Driver Log Dir). Additionally, this collects spark-shell driver logs at INFO level by default. The change is that instead of setting root logger level to WARN, we will set the consoleAppender threshold to WARN, in case of spark-shell. This ensures that only WARN logs are printed on CONSOLE but other log appenders still capture INFO (or the default log level logs). 1. Verified that logs are written to local and remote dir 2. Added a unit test case 3. Verified this for spark-shell, client mode and pyspark. 4. Verified in both non-kerberos and kerberos environment 5. Verified with following unexpected termination conditions: Ctrl + C, Driver OOM, Large Log Files 6. Ran an application in spark-shell and ensured that driver logs were captured at INFO level 7. Started the application at WARN level, programmatically changed the level to INFO and ensured that logs on console were printed at INFO level Closes #22504 from ankuriitg/ankurgupta/SPARK-25118. Authored-by: ankurgupta <ankur.gupta@cloudera.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-14 11:23:24 -05:00
<tr>
<td><code>spark.driver.log.dfsDir</code></td>
<td>(none)</td>
<td>
Base directory in which Spark driver logs are synced, if <code>spark.driver.log.persistToDfs.enabled</code>
is true. Within this base directory, each application logs the driver logs to an application specific file.
Users may want to set this to a unified location like an HDFS directory so driver log files can be persisted
for later usage. This directory should allow any Spark user to read/write files and the Spark History Server
user to delete files. Additionally, older logs from this directory are cleaned by the
<a href="monitoring.html#spark-history-server-configuration-options">Spark History Server</a> if
<code>spark.history.fs.driverlog.cleaner.enabled</code> is true and, if they are older than max age configured
by setting <code>spark.history.fs.driverlog.cleaner.maxAge</code>.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
[SPARK-25118][CORE] Persist Driver Logs in Client mode to Hdfs Currently, we do not have a mechanism to collect driver logs if a user chooses to run their application in client mode. This is a big issue as admin teams need to create their own mechanisms to capture driver logs. This commit adds a logger which, if enabled, adds a local log appender to the root logger and asynchronously syncs it an application specific log file on hdfs (Spark Driver Log Dir). Additionally, this collects spark-shell driver logs at INFO level by default. The change is that instead of setting root logger level to WARN, we will set the consoleAppender threshold to WARN, in case of spark-shell. This ensures that only WARN logs are printed on CONSOLE but other log appenders still capture INFO (or the default log level logs). 1. Verified that logs are written to local and remote dir 2. Added a unit test case 3. Verified this for spark-shell, client mode and pyspark. 4. Verified in both non-kerberos and kerberos environment 5. Verified with following unexpected termination conditions: Ctrl + C, Driver OOM, Large Log Files 6. Ran an application in spark-shell and ensured that driver logs were captured at INFO level 7. Started the application at WARN level, programmatically changed the level to INFO and ensured that logs on console were printed at INFO level Closes #22504 from ankuriitg/ankurgupta/SPARK-25118. Authored-by: ankurgupta <ankur.gupta@cloudera.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-14 11:23:24 -05:00
</tr>
<tr>
<td><code>spark.driver.log.persistToDfs.enabled</code></td>
<td>false</td>
<td>
If true, spark application running in client mode will write driver logs to a persistent storage, configured
in <code>spark.driver.log.dfsDir</code>. If <code>spark.driver.log.dfsDir</code> is not configured, driver logs
will not be persisted. Additionally, enable the cleaner by setting <code>spark.history.fs.driverlog.cleaner.enabled</code>
to true in <a href="monitoring.html#spark-history-server-configuration-options">Spark History Server</a>.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
[SPARK-25118][CORE] Persist Driver Logs in Client mode to Hdfs Currently, we do not have a mechanism to collect driver logs if a user chooses to run their application in client mode. This is a big issue as admin teams need to create their own mechanisms to capture driver logs. This commit adds a logger which, if enabled, adds a local log appender to the root logger and asynchronously syncs it an application specific log file on hdfs (Spark Driver Log Dir). Additionally, this collects spark-shell driver logs at INFO level by default. The change is that instead of setting root logger level to WARN, we will set the consoleAppender threshold to WARN, in case of spark-shell. This ensures that only WARN logs are printed on CONSOLE but other log appenders still capture INFO (or the default log level logs). 1. Verified that logs are written to local and remote dir 2. Added a unit test case 3. Verified this for spark-shell, client mode and pyspark. 4. Verified in both non-kerberos and kerberos environment 5. Verified with following unexpected termination conditions: Ctrl + C, Driver OOM, Large Log Files 6. Ran an application in spark-shell and ensured that driver logs were captured at INFO level 7. Started the application at WARN level, programmatically changed the level to INFO and ensured that logs on console were printed at INFO level Closes #22504 from ankuriitg/ankurgupta/SPARK-25118. Authored-by: ankurgupta <ankur.gupta@cloudera.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-14 11:23:24 -05:00
</tr>
<tr>
<td><code>spark.driver.log.layout</code></td>
<td>%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n</td>
<td>
The layout for the driver logs that are synced to <code>spark.driver.log.dfsDir</code>. If this is not configured,
it uses the layout for the first appender defined in log4j.properties. If that is also not configured, driver logs
use the default layout.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
[SPARK-25118][CORE] Persist Driver Logs in Client mode to Hdfs Currently, we do not have a mechanism to collect driver logs if a user chooses to run their application in client mode. This is a big issue as admin teams need to create their own mechanisms to capture driver logs. This commit adds a logger which, if enabled, adds a local log appender to the root logger and asynchronously syncs it an application specific log file on hdfs (Spark Driver Log Dir). Additionally, this collects spark-shell driver logs at INFO level by default. The change is that instead of setting root logger level to WARN, we will set the consoleAppender threshold to WARN, in case of spark-shell. This ensures that only WARN logs are printed on CONSOLE but other log appenders still capture INFO (or the default log level logs). 1. Verified that logs are written to local and remote dir 2. Added a unit test case 3. Verified this for spark-shell, client mode and pyspark. 4. Verified in both non-kerberos and kerberos environment 5. Verified with following unexpected termination conditions: Ctrl + C, Driver OOM, Large Log Files 6. Ran an application in spark-shell and ensured that driver logs were captured at INFO level 7. Started the application at WARN level, programmatically changed the level to INFO and ensured that logs on console were printed at INFO level Closes #22504 from ankuriitg/ankurgupta/SPARK-25118. Authored-by: ankurgupta <ankur.gupta@cloudera.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-14 11:23:24 -05:00
</tr>
<tr>
<td><code>spark.driver.log.allowErasureCoding</code></td>
<td>false</td>
<td>
Whether to allow driver logs to use erasure coding. On HDFS, erasure coded files will not
update as quickly as regular replicated files, so they make take longer to reflect changes
written by the application. Note that even if this is true, Spark will still not force the
file to use erasure coding, it will simply use file system defaults.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
</tr>
</table>
Apart from these, the following properties are also available, and may be useful in some situations:
### Runtime Environment
<table class="table">
2020-02-26 20:57:34 -05:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.driver.extraClassPath</code></td>
<td>(none)</td>
<td>
Extra classpath entries to prepend to the classpath of the driver.
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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-class-path</code> command line option or in
your default properties file.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.driver.defaultJavaOptions</code></td>
<td>(none)</td>
<td>
A string of default JVM options to prepend to <code>spark.driver.extraJavaOptions</code>.
This is intended to be set by administrators.
For instance, GC settings or other logging.
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.
<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-java-options</code> command line option or in
your default properties file.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.driver.extraJavaOptions</code></td>
<td>(none)</td>
<td>
A string of extra JVM options to pass to the driver. This is intended to be set by users.
For instance, GC settings or other logging.
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.
<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-java-options</code> command line option or in
your default properties file.
<code>spark.driver.defaultJavaOptions</code> will be prepended to this configuration.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</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.
<br /><em>Note:</em> In client mode, this config must not be set through the <code>SparkConf</code>
directly in your application, because the driver JVM has already started at that point.
Instead, please set this through the <code>--driver-library-path</code> command line option or in
your default properties file.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</td>
</tr>
[SPARK-2996] Implement userClassPathFirst for driver, yarn. Yarn's config option `spark.yarn.user.classpath.first` does not work the same way as `spark.files.userClassPathFirst`; Yarn's version is a lot more dangerous, in that it modifies the system classpath, instead of restricting the changes to the user's class loader. So this change implements the behavior of the latter for Yarn, and deprecates the more dangerous choice. To be able to achieve feature-parity, I also implemented the option for drivers (the existing option only applies to executors). So now there are two options, each controlling whether to apply userClassPathFirst to the driver or executors. The old option was deprecated, and aliased to the new one (`spark.executor.userClassPathFirst`). The existing "child-first" class loader also had to be fixed. It didn't handle resources, and it was also doing some things that ended up causing JVM errors depending on how things were being called. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #3233 from vanzin/SPARK-2996 and squashes the following commits: 9cf9cf1 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 a1499e2 [Marcelo Vanzin] Remove SPARK_HOME propagation. fa7df88 [Marcelo Vanzin] Remove 'test.resource' file, create it dynamically. a8c69f1 [Marcelo Vanzin] Review feedback. cabf962 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 a1b8d7e [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 3f768e3 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 2ce3c7a [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 0e6d6be [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 70d4044 [Marcelo Vanzin] Fix pyspark/yarn-cluster test. 0fe7777 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 0e6ef19 [Marcelo Vanzin] Move class loaders around and make names more meaninful. fe970a7 [Marcelo Vanzin] Review feedback. 25d4fed [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 3cb6498 [Marcelo Vanzin] Call the right loadClass() method on the parent. fbb8ab5 [Marcelo Vanzin] Add locking in loadClass() to avoid deadlocks. 2e6c4b7 [Marcelo Vanzin] Mention new setting in documentation. b6497f9 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 a10f379 [Marcelo Vanzin] Some feedback. 3730151 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 f513871 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 44010b6 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 7b57cba [Marcelo Vanzin] Remove now outdated message. 5304d64 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 35949c8 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 54e1a98 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 d1273b2 [Marcelo Vanzin] Add test file to rat exclude. fa1aafa [Marcelo Vanzin] Remove write check on user jars. 89d8072 [Marcelo Vanzin] Cleanups. a963ea3 [Marcelo Vanzin] Implement spark.driver.userClassPathFirst for standalone cluster mode. 50afa5f [Marcelo Vanzin] Fix Yarn executor command line. 7d14397 [Marcelo Vanzin] Register user jars in executor up front. 7f8603c [Marcelo Vanzin] Fix yarn-cluster mode without userClassPathFirst. 20373f5 [Marcelo Vanzin] Fix ClientBaseSuite. 55c88fa [Marcelo Vanzin] Run all Yarn integration tests via spark-submit. 0b64d92 [Marcelo Vanzin] Add deprecation warning to yarn option. 4a84d87 [Marcelo Vanzin] Fix the child-first class loader. d0394b8 [Marcelo Vanzin] Add "deprecated configs" to SparkConf. 46d8cf2 [Marcelo Vanzin] Update doc with new option, change name to "userClassPathFirst". a314f2d [Marcelo Vanzin] Enable driver class path isolation in SparkSubmit. 91f7e54 [Marcelo Vanzin] [yarn] Enable executor class path isolation. a853e74 [Marcelo Vanzin] Re-work CoarseGrainedExecutorBackend command line arguments. 89522ef [Marcelo Vanzin] Add class path isolation support for Yarn cluster mode.
2015-02-10 00:17:06 -05:00
<tr>
<td><code>spark.driver.userClassPathFirst</code></td>
<td>false</td>
<td>
(Experimental) Whether to give user-added jars precedence over Spark's own jars when loading
classes in the 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.
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>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.3.0</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
</tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.executor.extraClassPath</code></td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td>(none)</td>
<td>
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
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
this option.
</td>
[SPARK-30911][CORE][DOC] Add version information to the configuration of Status ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Status`. 2.Update the docs of `Status`. 3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847 I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |   spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |   spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |   spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27848 from beliefer/add-version-to-status-config. Lead-authored-by: beliefer <beliefer@163.com> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 22:03:47 -04:00
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.executor.defaultJavaOptions</code></td>
<td>(none)</td>
<td>
A string of default JVM options to prepend to <code>spark.executor.extraJavaOptions</code>.
This is intended to be set by administrators.
For instance, GC settings or other logging.
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.
The following symbols, if present will be interpolated: {{APP_ID}} will be replaced by
application ID and {{EXECUTOR_ID}} will be replaced by executor ID. For example, to enable
verbose gc logging to a file named for the executor ID of the app in /tmp, pass a 'value' of:
<code>-verbose:gc -Xloggc:/tmp/{{APP_ID}}-{{EXECUTOR_ID}}.gc</code>
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.executor.extraJavaOptions</code></td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td>(none)</td>
<td>
A string of extra JVM options to pass to executors. This is intended to be set by users.
For instance, GC settings or other logging.
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.
The following symbols, if present will be interpolated: {{APP_ID}} will be replaced by
application ID and {{EXECUTOR_ID}} will be replaced by executor ID. For example, to enable
verbose gc logging to a file named for the executor ID of the app in /tmp, pass a 'value' of:
<code>-verbose:gc -Xloggc:/tmp/{{APP_ID}}-{{EXECUTOR_ID}}.gc</code>
<code>spark.executor.defaultJavaOptions</code> will be prepended to this configuration.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</td>
</tr>
<tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td><code>spark.executor.extraLibraryPath</code></td>
<td>(none)</td>
<td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
Set a special library path to use when launching executor JVM's.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</td>
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.1.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>2.0.2</td>
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<tr>
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
2015-04-28 15:18:55 -04:00
<td><code>spark.executor.logs.rolling.maxSize</code></td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td>(none)</td>
<td>
Set the max size of the file in bytes by which the executor logs will be rolled over.
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
2015-04-28 15:18:55 -04:00
Rolling is disabled by default. See <code>spark.executor.logs.rolling.maxRetainedFiles</code>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
for automatic cleaning of old logs.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.4.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
<tr>
<td><code>spark.executor.logs.rolling.strategy</code></td>
<td>(none)</td>
<td>
Set the strategy of rolling of executor logs. By default it is disabled. It can
be set to "time" (time-based rolling) or "size" (size-based rolling). For "time",
use <code>spark.executor.logs.rolling.time.interval</code> to set the rolling interval.
For "size", use <code>spark.executor.logs.rolling.maxSize</code> to set
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
the maximum file size for rolling.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.1.0</td>
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
</tr>
<tr>
<td><code>spark.executor.logs.rolling.time.interval</code></td>
<td>daily</td>
<td>
Set the time interval by which the executor logs will be rolled over.
Rolling is disabled by default. Valid values are <code>daily</code>, <code>hourly</code>, <code>minutely</code> or
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.1.0</td>
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
</tr>
<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>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td>false</td>
<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.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.3.0</td>
</tr>
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.executorEnv.[EnvironmentVariableName]</code></td>
<td>(none)</td>
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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.
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>0.9.0</td>
[SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu <davies.liu@gmail.com> Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-25 01:53:47 -04:00
</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|token</td>
[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
<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 key or
value, the value is redacted from the environment UI and various logs like YARN and event logs.
[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
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.1.2</td>
[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>
[SPARK-3478] [PySpark] Profile the Python tasks This patch add profiling support for PySpark, it will show the profiling results before the driver exits, here is one example: ``` ============================================================ Profile of RDD<id=3> ============================================================ 5146507 function calls (5146487 primitive calls) in 71.094 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge) 20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__) 20 0.017 0.001 0.017 0.001 {cPickle.dumps} 1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>) 20 0.001 0.000 0.001 0.000 {reduce} 21 0.001 0.000 0.001 0.000 {cPickle.loads} 20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames) 41 0.001 0.000 0.001 0.000 serializers.py:461(read_int) 40 0.001 0.000 0.002 0.000 serializers.py:179(_batched) 62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects} 20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>) 20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream) 40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func) 41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream) 40 0.000 0.000 71.072 1.777 rdd.py:304(func) 20 0.000 0.000 71.094 3.555 worker.py:82(process) ``` Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk by `sc.dump_profiles(path)`, such as ```python >>> sc._conf.set("spark.python.profile", "true") >>> rdd = sc.parallelize(range(100)).map(str) >>> rdd.count() 100 >>> sc.show_profiles() ============================================================ Profile of RDD<id=1> ============================================================ 284 function calls (276 primitive calls) in 0.001 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream) 4 0.000 0.000 0.000 0.000 {reduce} 12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func) 4 0.000 0.000 0.000 0.000 {cPickle.loads} 4 0.000 0.000 0.000 0.000 {cPickle.dumps} 104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>) 8 0.000 0.000 0.000 0.000 serializers.py:461(read_int) 12 0.000 0.000 0.000 0.000 rdd.py:303(func) ``` The profiling is disabled by default, can be enabled by "spark.python.profile=true". Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump" This is bugfix of #2351 cc JoshRosen Author: Davies Liu <davies.liu@gmail.com> Closes #2556 from davies/profiler and squashes the following commits: e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 858e74c [Davies Liu] compatitable with python 2.6 7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles() 2b0daf2 [Davies Liu] fix docs 7a56c24 [Davies Liu] bugfix cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 09d02c3 [Davies Liu] Merge branch 'master' into profiler c23865c [Davies Liu] Merge branch 'master' into profiler 15d6f18 [Davies Liu] add docs for two configs dadee1a [Davies Liu] add docs string and clear profiles after show or dump 4f8309d [Davies Liu] address comment, add tests 0a5b6eb [Davies Liu] fix Python UDF 4b20494 [Davies Liu] add profile for python
2014-09-30 21:24:57 -04:00
<tr>
<td><code>spark.python.profile</code></td>
<td>false</td>
<td>
Enable profiling in Python worker, the profile result will show up by <code>sc.show_profiles()</code>,
or it will be displayed before the driver exits. It also can be dumped into disk by
<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.
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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.2.0</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>
<tr>
<td><code>spark.python.profile.dump</code></td>
<td>(none)</td>
<td>
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
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 <code>pstats.Stats()</code>. If this is specified, the profile result will not be displayed
[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
automatically.
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.2.0</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>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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 with a size unit suffix ("k", "m", "g" or "t")
(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.
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.1.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
[SPARK-3030] [PySpark] Reuse Python worker Reuse Python worker to avoid the overhead of fork() Python process for each tasks. It also tracks the broadcasts for each worker, avoid sending repeated broadcasts. This can reduce the time for dummy task from 22ms to 13ms (-40%). It can help to reduce the latency for Spark Streaming. For a job with broadcast (43M after compress): ``` b = sc.broadcast(set(range(30000000))) print sc.parallelize(range(24000), 100).filter(lambda x: x in b.value).count() ``` It will finish in 281s without reused worker, and it will finish in 65s with reused worker(4 CPUs). After reusing the worker, it can save about 9 seconds for transfer and deserialize the broadcast for each tasks. It's enabled by default, could be disabled by `spark.python.worker.reuse = false`. Author: Davies Liu <davies.liu@gmail.com> Closes #2259 from davies/reuse-worker and squashes the following commits: f11f617 [Davies Liu] Merge branch 'master' into reuse-worker 3939f20 [Davies Liu] fix bug in serializer in mllib cf1c55e [Davies Liu] address comments 3133a60 [Davies Liu] fix accumulator with reused worker 760ab1f [Davies Liu] do not reuse worker if there are any exceptions 7abb224 [Davies Liu] refactor: sychronized with itself ac3206e [Davies Liu] renaming 8911f44 [Davies Liu] synchronized getWorkerBroadcasts() 6325fc1 [Davies Liu] bugfix: bid >= 0 e0131a2 [Davies Liu] fix name of config 583716e [Davies Liu] only reuse completed and not interrupted worker ace2917 [Davies Liu] kill python worker after timeout 6123d0f [Davies Liu] track broadcasts for each worker 8d2f08c [Davies Liu] reuse python worker
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 task. It will be very useful
if there is a large broadcast, then the broadcast will not need to be transferred
[SPARK-3030] [PySpark] Reuse Python worker Reuse Python worker to avoid the overhead of fork() Python process for each tasks. It also tracks the broadcasts for each worker, avoid sending repeated broadcasts. This can reduce the time for dummy task from 22ms to 13ms (-40%). It can help to reduce the latency for Spark Streaming. For a job with broadcast (43M after compress): ``` b = sc.broadcast(set(range(30000000))) print sc.parallelize(range(24000), 100).filter(lambda x: x in b.value).count() ``` It will finish in 281s without reused worker, and it will finish in 65s with reused worker(4 CPUs). After reusing the worker, it can save about 9 seconds for transfer and deserialize the broadcast for each tasks. It's enabled by default, could be disabled by `spark.python.worker.reuse = false`. Author: Davies Liu <davies.liu@gmail.com> Closes #2259 from davies/reuse-worker and squashes the following commits: f11f617 [Davies Liu] Merge branch 'master' into reuse-worker 3939f20 [Davies Liu] fix bug in serializer in mllib cf1c55e [Davies Liu] address comments 3133a60 [Davies Liu] fix accumulator with reused worker 760ab1f [Davies Liu] do not reuse worker if there are any exceptions 7abb224 [Davies Liu] refactor: sychronized with itself ac3206e [Davies Liu] renaming 8911f44 [Davies Liu] synchronized getWorkerBroadcasts() 6325fc1 [Davies Liu] bugfix: bid >= 0 e0131a2 [Davies Liu] fix name of config 583716e [Davies Liu] only reuse completed and not interrupted worker ace2917 [Davies Liu] kill python worker after timeout 6123d0f [Davies Liu] track broadcasts for each worker 8d2f08c [Davies Liu] reuse python worker
2014-09-13 19:22:04 -04:00
from JVM to Python worker for every task.
</td>
2020-02-26 20:57:34 -05:00
<td>1.2.0</td>
[SPARK-3030] [PySpark] Reuse Python worker Reuse Python worker to avoid the overhead of fork() Python process for each tasks. It also tracks the broadcasts for each worker, avoid sending repeated broadcasts. This can reduce the time for dummy task from 22ms to 13ms (-40%). It can help to reduce the latency for Spark Streaming. For a job with broadcast (43M after compress): ``` b = sc.broadcast(set(range(30000000))) print sc.parallelize(range(24000), 100).filter(lambda x: x in b.value).count() ``` It will finish in 281s without reused worker, and it will finish in 65s with reused worker(4 CPUs). After reusing the worker, it can save about 9 seconds for transfer and deserialize the broadcast for each tasks. It's enabled by default, could be disabled by `spark.python.worker.reuse = false`. Author: Davies Liu <davies.liu@gmail.com> Closes #2259 from davies/reuse-worker and squashes the following commits: f11f617 [Davies Liu] Merge branch 'master' into reuse-worker 3939f20 [Davies Liu] fix bug in serializer in mllib cf1c55e [Davies Liu] address comments 3133a60 [Davies Liu] fix accumulator with reused worker 760ab1f [Davies Liu] do not reuse worker if there are any exceptions 7abb224 [Davies Liu] refactor: sychronized with itself ac3206e [Davies Liu] renaming 8911f44 [Davies Liu] synchronized getWorkerBroadcasts() 6325fc1 [Davies Liu] bugfix: bid >= 0 e0131a2 [Davies Liu] fix name of config 583716e [Davies Liu] only reuse completed and not interrupted worker ace2917 [Davies Liu] kill python worker after timeout 6123d0f [Davies Liu] track broadcasts for each worker 8d2f08c [Davies Liu] reuse python worker
2014-09-13 19:22:04 -04:00
</tr>
<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. Globs are allowed.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.0.0</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. Globs are allowed.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.0.1</td>
</tr>
<tr>
<td><code>spark.jars</code></td>
<td></td>
<td>
Comma-separated list of jars to include on the driver and executor classpaths. Globs are allowed.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.9.0</td>
</tr>
<tr>
<td><code>spark.jars.packages</code></td>
<td></td>
<td>
Comma-separated list of Maven coordinates of jars to include on the driver and executor
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>.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.5.0</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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.5.0</td>
</tr>
<tr>
<td><code>spark.jars.ivy</code></td>
<td></td>
<td>
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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.3.0</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> or <code>spark.jars.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
[SPARK-28609][DOC] Fix broken styles/links and make up-to-date ## What changes were proposed in this pull request? This PR aims to fix the broken styles/links and make the doc up-to-date for Apache Spark 2.4.4 and 3.0.0 release. - `building-spark.md` ![Screen Shot 2019-08-02 at 10 33 51 PM](https://user-images.githubusercontent.com/9700541/62407962-a248ec80-b575-11e9-8a16-532e9bc421f8.png) - `configuration.md` ![Screen Shot 2019-08-02 at 10 34 52 PM](https://user-images.githubusercontent.com/9700541/62407969-c7d5f600-b575-11e9-9b1a-a76c6cc095c5.png) - `sql-pyspark-pandas-with-arrow.md` ![Screen Shot 2019-08-02 at 10 36 14 PM](https://user-images.githubusercontent.com/9700541/62407979-18e5ea00-b576-11e9-99af-7ad9264656ae.png) - `streaming-programming-guide.md` ![Screen Shot 2019-08-02 at 10 37 11 PM](https://user-images.githubusercontent.com/9700541/62407981-213e2500-b576-11e9-8bc5-a925df7e98a7.png) - `structured-streaming-programming-guide.md` (1/2) ![Screen Shot 2019-08-02 at 10 38 20 PM](https://user-images.githubusercontent.com/9700541/62408001-49c61f00-b576-11e9-9519-f699775ceecd.png) - `structured-streaming-programming-guide.md` (2/2) ![Screen Shot 2019-08-02 at 10 40 05 PM](https://user-images.githubusercontent.com/9700541/62408017-7f6b0800-b576-11e9-9341-52664ba6b460.png) - `submitting-applications.md` ![Screen Shot 2019-08-02 at 10 41 13 PM](https://user-images.githubusercontent.com/9700541/62408027-b2ad9700-b576-11e9-910e-8f22173e1251.png) ## How was this patch tested? Manual. Build the doc. ``` SKIP_API=1 jekyll build ``` Closes #25345 from dongjoon-hyun/SPARK-28609. Authored-by: Dongjoon Hyun <dhyun@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-04 12:42:47 -04:00
found at <a href="http://ant.apache.org/ivy/history/latest-milestone/settings.html">Settings Files</a>
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>2.2.0</td>
</tr>
<tr>
<td><code>spark.jars.repositories</code></td>
<td></td>
<td>
Comma-separated list of additional remote repositories to search for the maven coordinates
given with <code>--packages</code> or <code>spark.jars.packages</code>.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>2.3.0</td>
</tr>
<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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.1.0</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.1.0</td>
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</table>
### Shuffle Behavior
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<table class="table">
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
2015-04-28 15:18:55 -04:00
<td><code>spark.reducer.maxSizeInFlight</code></td>
<td>48m</td>
<td>
Maximum size of map outputs to fetch simultaneously from each reduce task, in MiB unless
otherwise specified. Since 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.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.4.0</td>
</tr>
<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 inbound 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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>2.0.0</td>
</tr>
<tr>
<td><code>spark.reducer.maxBlocksInFlightPerAddress</code></td>
<td>Int.MaxValue</td>
<td>
This configuration limits the number of remote blocks being fetched per reduce task from a
given host port. When a large number of blocks are being requested from a given address in a
single fetch or simultaneously, this could crash the serving executor or Node Manager. This
is especially useful to reduce the load on the Node Manager when external shuffle is enabled.
You can mitigate this issue by setting it to a lower value.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.2.1</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.shuffle.compress</code></td>
<td>true</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>0.6.0</td>
</tr>
SPARK-1305: Support persisting RDD's directly to Tachyon Move the PR#468 of apache-incubator-spark to the apache-spark "Adding an option to persist Spark RDD blocks into Tachyon." Author: Haoyuan Li <haoyuan@cs.berkeley.edu> Author: RongGu <gurongwalker@gmail.com> Closes #158 from RongGu/master and squashes the following commits: 72b7768 [Haoyuan Li] merge master 9f7fa1b [Haoyuan Li] fix code style ae7834b [Haoyuan Li] minor cleanup a8b3ec6 [Haoyuan Li] merge master branch e0f4891 [Haoyuan Li] better check offheap. 55b5918 [RongGu] address matei's comment on the replication of offHeap storagelevel 7cd4600 [RongGu] remove some logic code for tachyonstore's replication 51149e7 [RongGu] address aaron's comment on returning value of the remove() function in tachyonstore 8adfcfa [RongGu] address arron's comment on inTachyonSize 120e48a [RongGu] changed the root-level dir name in Tachyon 5cc041c [Haoyuan Li] address aaron's comments 9b97935 [Haoyuan Li] address aaron's comments d9a6438 [Haoyuan Li] fix for pspark 77d2703 [Haoyuan Li] change python api.git status 3dcace4 [Haoyuan Li] address matei's comments 91fa09d [Haoyuan Li] address patrick's comments 589eafe [Haoyuan Li] use TRY_CACHE instead of MUST_CACHE 64348b2 [Haoyuan Li] update conf docs. ed73e19 [Haoyuan Li] Merge branch 'master' of github.com:RongGu/spark-1 619a9a8 [RongGu] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore be79d77 [RongGu] find a way to clean up some unnecessay metods and classed to make the code simpler 49cc724 [Haoyuan Li] update docs with off_headp option 4572f9f [RongGu] reserving the old apply function API of StorageLevel 04301d3 [RongGu] rename StorageLevel.TACHYON to Storage.OFF_HEAP c9aeabf [RongGu] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP 76805aa [RongGu] unifies the config properties name prefix; add the configs into docs/configuration.md e700d9c [RongGu] add the SparkTachyonHdfsLR example and some comments fd84156 [RongGu] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix 939e467 [Haoyuan Li] 0.4.1-thrift from maven central 86a2eab [Haoyuan Li] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1 16c5798 [RongGu] make the dependency on tachyon as tachyon-0.4.1-thrift eacb2e8 [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 bbeb4de [RongGu] fix the JsonProtocolSuite test failure problem 6adb58f [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 d827250 [RongGu] fix JsonProtocolSuie test failure 716e93b [Haoyuan Li] revert the version ca14469 [Haoyuan Li] bump tachyon version to 0.4.1-thrift 2825a13 [RongGu] up-merging to the current master branch of the apache spark 6a22c1a [Haoyuan Li] fix scalastyle 8968b67 [Haoyuan Li] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client. 77be7e8 [RongGu] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice. 1dcadf9 [Haoyuan Li] typo bf278fa [Haoyuan Li] fix python tests e82909c [Haoyuan Li] minor cleanup 776a56c [Haoyuan Li] address patrick's and ali's comments from the previous PR 8859371 [Haoyuan Li] various minor fixes and clean up e3ddbba [Haoyuan Li] add doc to use Tachyon cache mode. fcaeab2 [Haoyuan Li] address Aaron's comment e554b1e [Haoyuan Li] add python code 47304b3 [Haoyuan Li] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels. dc8ef24 [Haoyuan Li] add old storelevel constructor e01a271 [Haoyuan Li] update tachyon 0.4.1 8011a96 [RongGu] fix a brought-in mistake in StorageLevel 70ca182 [RongGu] a bit change in comment 556978b [RongGu] fix the scalastyle errors 791189b [RongGu] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark
2014-04-04 23:36:24 -04:00
<tr>
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
2015-04-28 15:18:55 -04:00
<td><code>spark.shuffle.file.buffer</code></td>
<td>32k</td>
SPARK-1305: Support persisting RDD's directly to Tachyon Move the PR#468 of apache-incubator-spark to the apache-spark "Adding an option to persist Spark RDD blocks into Tachyon." Author: Haoyuan Li <haoyuan@cs.berkeley.edu> Author: RongGu <gurongwalker@gmail.com> Closes #158 from RongGu/master and squashes the following commits: 72b7768 [Haoyuan Li] merge master 9f7fa1b [Haoyuan Li] fix code style ae7834b [Haoyuan Li] minor cleanup a8b3ec6 [Haoyuan Li] merge master branch e0f4891 [Haoyuan Li] better check offheap. 55b5918 [RongGu] address matei's comment on the replication of offHeap storagelevel 7cd4600 [RongGu] remove some logic code for tachyonstore's replication 51149e7 [RongGu] address aaron's comment on returning value of the remove() function in tachyonstore 8adfcfa [RongGu] address arron's comment on inTachyonSize 120e48a [RongGu] changed the root-level dir name in Tachyon 5cc041c [Haoyuan Li] address aaron's comments 9b97935 [Haoyuan Li] address aaron's comments d9a6438 [Haoyuan Li] fix for pspark 77d2703 [Haoyuan Li] change python api.git status 3dcace4 [Haoyuan Li] address matei's comments 91fa09d [Haoyuan Li] address patrick's comments 589eafe [Haoyuan Li] use TRY_CACHE instead of MUST_CACHE 64348b2 [Haoyuan Li] update conf docs. ed73e19 [Haoyuan Li] Merge branch 'master' of github.com:RongGu/spark-1 619a9a8 [RongGu] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore be79d77 [RongGu] find a way to clean up some unnecessay metods and classed to make the code simpler 49cc724 [Haoyuan Li] update docs with off_headp option 4572f9f [RongGu] reserving the old apply function API of StorageLevel 04301d3 [RongGu] rename StorageLevel.TACHYON to Storage.OFF_HEAP c9aeabf [RongGu] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP 76805aa [RongGu] unifies the config properties name prefix; add the configs into docs/configuration.md e700d9c [RongGu] add the SparkTachyonHdfsLR example and some comments fd84156 [RongGu] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix 939e467 [Haoyuan Li] 0.4.1-thrift from maven central 86a2eab [Haoyuan Li] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1 16c5798 [RongGu] make the dependency on tachyon as tachyon-0.4.1-thrift eacb2e8 [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 bbeb4de [RongGu] fix the JsonProtocolSuite test failure problem 6adb58f [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 d827250 [RongGu] fix JsonProtocolSuie test failure 716e93b [Haoyuan Li] revert the version ca14469 [Haoyuan Li] bump tachyon version to 0.4.1-thrift 2825a13 [RongGu] up-merging to the current master branch of the apache spark 6a22c1a [Haoyuan Li] fix scalastyle 8968b67 [Haoyuan Li] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client. 77be7e8 [RongGu] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice. 1dcadf9 [Haoyuan Li] typo bf278fa [Haoyuan Li] fix python tests e82909c [Haoyuan Li] minor cleanup 776a56c [Haoyuan Li] address patrick's and ali's comments from the previous PR 8859371 [Haoyuan Li] various minor fixes and clean up e3ddbba [Haoyuan Li] add doc to use Tachyon cache mode. fcaeab2 [Haoyuan Li] address Aaron's comment e554b1e [Haoyuan Li] add python code 47304b3 [Haoyuan Li] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels. dc8ef24 [Haoyuan Li] add old storelevel constructor e01a271 [Haoyuan Li] update tachyon 0.4.1 8011a96 [RongGu] fix a brought-in mistake in StorageLevel 70ca182 [RongGu] a bit change in comment 556978b [RongGu] fix the scalastyle errors 791189b [RongGu] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark
2014-04-04 23:36:24 -04:00
<td>
Size of the in-memory buffer for each shuffle file output stream, in KiB unless otherwise
specified. These buffers reduce the number of disk seeks and system calls made in creating
intermediate shuffle files.
SPARK-1305: Support persisting RDD's directly to Tachyon Move the PR#468 of apache-incubator-spark to the apache-spark "Adding an option to persist Spark RDD blocks into Tachyon." Author: Haoyuan Li <haoyuan@cs.berkeley.edu> Author: RongGu <gurongwalker@gmail.com> Closes #158 from RongGu/master and squashes the following commits: 72b7768 [Haoyuan Li] merge master 9f7fa1b [Haoyuan Li] fix code style ae7834b [Haoyuan Li] minor cleanup a8b3ec6 [Haoyuan Li] merge master branch e0f4891 [Haoyuan Li] better check offheap. 55b5918 [RongGu] address matei's comment on the replication of offHeap storagelevel 7cd4600 [RongGu] remove some logic code for tachyonstore's replication 51149e7 [RongGu] address aaron's comment on returning value of the remove() function in tachyonstore 8adfcfa [RongGu] address arron's comment on inTachyonSize 120e48a [RongGu] changed the root-level dir name in Tachyon 5cc041c [Haoyuan Li] address aaron's comments 9b97935 [Haoyuan Li] address aaron's comments d9a6438 [Haoyuan Li] fix for pspark 77d2703 [Haoyuan Li] change python api.git status 3dcace4 [Haoyuan Li] address matei's comments 91fa09d [Haoyuan Li] address patrick's comments 589eafe [Haoyuan Li] use TRY_CACHE instead of MUST_CACHE 64348b2 [Haoyuan Li] update conf docs. ed73e19 [Haoyuan Li] Merge branch 'master' of github.com:RongGu/spark-1 619a9a8 [RongGu] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore be79d77 [RongGu] find a way to clean up some unnecessay metods and classed to make the code simpler 49cc724 [Haoyuan Li] update docs with off_headp option 4572f9f [RongGu] reserving the old apply function API of StorageLevel 04301d3 [RongGu] rename StorageLevel.TACHYON to Storage.OFF_HEAP c9aeabf [RongGu] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP 76805aa [RongGu] unifies the config properties name prefix; add the configs into docs/configuration.md e700d9c [RongGu] add the SparkTachyonHdfsLR example and some comments fd84156 [RongGu] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix 939e467 [Haoyuan Li] 0.4.1-thrift from maven central 86a2eab [Haoyuan Li] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1 16c5798 [RongGu] make the dependency on tachyon as tachyon-0.4.1-thrift eacb2e8 [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 bbeb4de [RongGu] fix the JsonProtocolSuite test failure problem 6adb58f [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 d827250 [RongGu] fix JsonProtocolSuie test failure 716e93b [Haoyuan Li] revert the version ca14469 [Haoyuan Li] bump tachyon version to 0.4.1-thrift 2825a13 [RongGu] up-merging to the current master branch of the apache spark 6a22c1a [Haoyuan Li] fix scalastyle 8968b67 [Haoyuan Li] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client. 77be7e8 [RongGu] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice. 1dcadf9 [Haoyuan Li] typo bf278fa [Haoyuan Li] fix python tests e82909c [Haoyuan Li] minor cleanup 776a56c [Haoyuan Li] address patrick's and ali's comments from the previous PR 8859371 [Haoyuan Li] various minor fixes and clean up e3ddbba [Haoyuan Li] add doc to use Tachyon cache mode. fcaeab2 [Haoyuan Li] address Aaron's comment e554b1e [Haoyuan Li] add python code 47304b3 [Haoyuan Li] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels. dc8ef24 [Haoyuan Li] add old storelevel constructor e01a271 [Haoyuan Li] update tachyon 0.4.1 8011a96 [RongGu] fix a brought-in mistake in StorageLevel 70ca182 [RongGu] a bit change in comment 556978b [RongGu] fix the scalastyle errors 791189b [RongGu] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark
2014-04-04 23:36:24 -04:00
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>1.4.0</td>
SPARK-1305: Support persisting RDD's directly to Tachyon Move the PR#468 of apache-incubator-spark to the apache-spark "Adding an option to persist Spark RDD blocks into Tachyon." Author: Haoyuan Li <haoyuan@cs.berkeley.edu> Author: RongGu <gurongwalker@gmail.com> Closes #158 from RongGu/master and squashes the following commits: 72b7768 [Haoyuan Li] merge master 9f7fa1b [Haoyuan Li] fix code style ae7834b [Haoyuan Li] minor cleanup a8b3ec6 [Haoyuan Li] merge master branch e0f4891 [Haoyuan Li] better check offheap. 55b5918 [RongGu] address matei's comment on the replication of offHeap storagelevel 7cd4600 [RongGu] remove some logic code for tachyonstore's replication 51149e7 [RongGu] address aaron's comment on returning value of the remove() function in tachyonstore 8adfcfa [RongGu] address arron's comment on inTachyonSize 120e48a [RongGu] changed the root-level dir name in Tachyon 5cc041c [Haoyuan Li] address aaron's comments 9b97935 [Haoyuan Li] address aaron's comments d9a6438 [Haoyuan Li] fix for pspark 77d2703 [Haoyuan Li] change python api.git status 3dcace4 [Haoyuan Li] address matei's comments 91fa09d [Haoyuan Li] address patrick's comments 589eafe [Haoyuan Li] use TRY_CACHE instead of MUST_CACHE 64348b2 [Haoyuan Li] update conf docs. ed73e19 [Haoyuan Li] Merge branch 'master' of github.com:RongGu/spark-1 619a9a8 [RongGu] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore be79d77 [RongGu] find a way to clean up some unnecessay metods and classed to make the code simpler 49cc724 [Haoyuan Li] update docs with off_headp option 4572f9f [RongGu] reserving the old apply function API of StorageLevel 04301d3 [RongGu] rename StorageLevel.TACHYON to Storage.OFF_HEAP c9aeabf [RongGu] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP 76805aa [RongGu] unifies the config properties name prefix; add the configs into docs/configuration.md e700d9c [RongGu] add the SparkTachyonHdfsLR example and some comments fd84156 [RongGu] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix 939e467 [Haoyuan Li] 0.4.1-thrift from maven central 86a2eab [Haoyuan Li] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1 16c5798 [RongGu] make the dependency on tachyon as tachyon-0.4.1-thrift eacb2e8 [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 bbeb4de [RongGu] fix the JsonProtocolSuite test failure problem 6adb58f [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 d827250 [RongGu] fix JsonProtocolSuie test failure 716e93b [Haoyuan Li] revert the version ca14469 [Haoyuan Li] bump tachyon version to 0.4.1-thrift 2825a13 [RongGu] up-merging to the current master branch of the apache spark 6a22c1a [Haoyuan Li] fix scalastyle 8968b67 [Haoyuan Li] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client. 77be7e8 [RongGu] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice. 1dcadf9 [Haoyuan Li] typo bf278fa [Haoyuan Li] fix python tests e82909c [Haoyuan Li] minor cleanup 776a56c [Haoyuan Li] address patrick's and ali's comments from the previous PR 8859371 [Haoyuan Li] various minor fixes and clean up e3ddbba [Haoyuan Li] add doc to use Tachyon cache mode. fcaeab2 [Haoyuan Li] address Aaron's comment e554b1e [Haoyuan Li] add python code 47304b3 [Haoyuan Li] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels. dc8ef24 [Haoyuan Li] add old storelevel constructor e01a271 [Haoyuan Li] update tachyon 0.4.1 8011a96 [RongGu] fix a brought-in mistake in StorageLevel 70ca182 [RongGu] a bit change in comment 556978b [RongGu] fix the scalastyle errors 791189b [RongGu] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark
2014-04-04 23:36:24 -04:00
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.shuffle.io.maxRetries</code></td>
<td>3</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.2.0</td>
</tr>
2013-02-27 01:52:38 -05:00
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.2.1</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.shuffle.io.preferDirectBufs</code></td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td>true</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.2.0</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
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.shuffle.io.retryWait</code></td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
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>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
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>.
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.2.1</td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</tr>
<tr>
<td><code>spark.shuffle.io.backLog</code></td>
<td>-1</td>
<td>
Length of the accept queue for the shuffle service. For large applications, this value may
need to be increased, so that incoming connections are not dropped if the service cannot keep
up with a large number of connections arriving in a short period of time. This needs to
be configured wherever the shuffle service itself is running, which may be outside of the
application (see <code>spark.shuffle.service.enabled</code> option below). If set below 1,
will fallback to OS default defined by Netty's <code>io.netty.util.NetUtil#SOMAXCONN</code>.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.1.1</td>
</tr>
<tr>
<td><code>spark.shuffle.service.enabled</code></td>
<td>false</td>
<td>
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
<code>spark.dynamicAllocation.enabled</code> is "true". The external shuffle service
must be set up in order to enable it. See
<a href="job-scheduling.html#configuration-and-setup">dynamic allocation
configuration and setup documentation</a> for more information.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>1.2.0</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>1.2.0</td>
</tr>
<tr>
<td><code>spark.shuffle.service.index.cache.size</code></td>
<td>100m</td>
<td>
Cache entries limited to the specified memory footprint, in bytes unless otherwise specified.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>2.3.0</td>
</tr>
<tr>
<td><code>spark.shuffle.maxChunksBeingTransferred</code></td>
<td>Long.MAX_VALUE</td>
<td>
The max number of chunks allowed to be transferred at the same time on shuffle service.
Note that new incoming connections will be closed when the max number is hit. The client will
retry according to the shuffle retry configs (see <code>spark.shuffle.io.maxRetries</code> and
<code>spark.shuffle.io.retryWait</code>), if those limits are reached the task will fail with
fetch failure.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>2.3.0</td>
</tr>
<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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>1.1.1</td>
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>0.9.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<tr>
<td><code>spark.shuffle.accurateBlockThreshold</code></td>
<td>100 * 1024 * 1024</td>
<td>
Threshold in bytes above which the size of shuffle blocks in HighlyCompressedMapStatus is
accurately recorded. This helps to prevent OOM by avoiding underestimating shuffle
block size when fetch shuffle blocks.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.2.1</td>
</tr>
<tr>
<td><code>spark.shuffle.registration.timeout</code></td>
<td>5000</td>
<td>
Timeout in milliseconds for registration to the external shuffle service.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.3.0</td>
</tr>
<tr>
<td><code>spark.shuffle.registration.maxAttempts</code></td>
<td>3</td>
<td>
When we fail to register to the external shuffle service, we will retry for maxAttempts times.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.3.0</td>
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</table>
### Spark UI
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<table class="table">
[SPARK-30889][SPARK-30913][CORE][DOC] Add version information to the configuration of Tests.scala and Worker ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Tests` and `Worker`. 2.Update the docs of `Worker`. I sorted out some information of `Tests` show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.testing.memory | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-395d07dcd46359cca610ce74357f0bb4 |   spark.testing.dynamicAllocation.scheduleInterval | 2.3.0 | SPARK-22864 | 4e9e6aee44bb2ddb41b567d659358b22fd824222#diff-b096353602813e47074ace09a3890d56 |   spark.testing | 1.0.1 | SPARK-1606 | ce57624b8232159fe3ec6db228afc622133df591#diff-d239aee594001f8391676e1047a0381e |   spark.test.noStageRetry | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.testing.reservedMemory | 1.6.0 | SPARK-12081 | 84c44b500b5c90dffbe1a6b0aa86f01699b09b96#diff-395d07dcd46359cca610ce74357f0bb4 | spark.testing.nHosts | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.testing.nExecutorsPerHost | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.testing.nCoresPerExecutor | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.resources.warnings.testing | 3.1.0 | SPARK-29148 | 496f6ac86001d284cbfb7488a63dd3a168919c0f#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.testing.resourceProfileManager | 3.1.0 | SPARK-29148 | 496f6ac86001d284cbfb7488a63dd3a168919c0f#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   I sorted out some information of `Worker` show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.worker.resourcesFile | 3.0.0 | SPARK-27369 | 7cbe01e8efc3f6cd3a0cac4bcfadea8fcc74a955#diff-b2fc8d6ab7ac5735085e2d6cfacb95da |   spark.worker.timeout | 0.6.2 | None | e395aa295aeec6767df798bf1002b1f30983c1cd#diff-776a630ac2b2ec5fe85c07ca20a58fc0 |   spark.worker.driverTerminateTimeout | 2.1.2 | SPARK-20843 | ebd72f453aa0b4f68760d28b3e93e6dd33856659#diff-829a8674171f92acd61007bedb1bfa4f |   spark.worker.cleanup.enabled | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.cleanup.interval | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.cleanup.appDataTtl | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.preferConfiguredMasterAddress | 2.2.1 | SPARK-20529 | 75e5ea294c15ecfb7366ae15dce196aa92c87ca4#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-48ca297b6536cb92362bec1487581f05 |   spark.worker.ui.retainedExecutors | 1.5.0 | SPARK-9202 | c0686668ae6a92b6bb4801a55c3b78aedbee816a#diff-916ca56b663f178f302c265b7ef38499 | spark.worker.ui.retainedDrivers | 1.5.0 | SPARK-9202 | c0686668ae6a92b6bb4801a55c3b78aedbee816a#diff-916ca56b663f178f302c265b7ef38499 | spark.worker.ui.compressedLogFileLengthCacheSize | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-d239aee594001f8391676e1047a0381e |   spark.worker.decommission.enabled | 3.1.0 | SPARK-20628 | d273a2bb0fac452a97f5670edd69d3e452e3e57e#diff-b2fc8d6ab7ac5735085e2d6cfacb95da |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27783 from beliefer/add-version-to-tests-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-04 21:58:21 -05:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td><code>spark.eventLog.logBlockUpdates.enabled</code></td>
<td>false</td>
<td>
Whether to log events for every block update, if <code>spark.eventLog.enabled</code> is true.
*Warning*: This will increase the size of the event log considerably.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>2.3.0</td>
</tr>
<tr>
<td><code>spark.eventLog.longForm.enabled</code></td>
<td>false</td>
<td>
If true, use the long form of call sites in the event log. Otherwise use the short form.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>2.4.0</td>
</tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.eventLog.compress</code></td>
<td>false</td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
Whether to compress logged events, if <code>spark.eventLog.enabled</code> is true.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.eventLog.compression.codec</code></td>
<td></td>
<td>
The codec to compress logged events. If this is not given,
<code>spark.io.compression.codec</code> will be used.
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>3.0.0</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
</tr>
<tr>
<td><code>spark.eventLog.erasureCoding.enabled</code></td>
<td>false</td>
<td>
Whether to allow event logs to use erasure coding, or turn erasure coding off, regardless of
filesystem defaults. On HDFS, erasure coded files will not update as quickly as regular
replicated files, so the application updates will take longer to appear in the History Server.
Note that even if this is true, Spark will still not force the file to use erasure coding, it
will simply use filesystem defaults.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
</tr>
SPARK-1305: Support persisting RDD's directly to Tachyon Move the PR#468 of apache-incubator-spark to the apache-spark "Adding an option to persist Spark RDD blocks into Tachyon." Author: Haoyuan Li <haoyuan@cs.berkeley.edu> Author: RongGu <gurongwalker@gmail.com> Closes #158 from RongGu/master and squashes the following commits: 72b7768 [Haoyuan Li] merge master 9f7fa1b [Haoyuan Li] fix code style ae7834b [Haoyuan Li] minor cleanup a8b3ec6 [Haoyuan Li] merge master branch e0f4891 [Haoyuan Li] better check offheap. 55b5918 [RongGu] address matei's comment on the replication of offHeap storagelevel 7cd4600 [RongGu] remove some logic code for tachyonstore's replication 51149e7 [RongGu] address aaron's comment on returning value of the remove() function in tachyonstore 8adfcfa [RongGu] address arron's comment on inTachyonSize 120e48a [RongGu] changed the root-level dir name in Tachyon 5cc041c [Haoyuan Li] address aaron's comments 9b97935 [Haoyuan Li] address aaron's comments d9a6438 [Haoyuan Li] fix for pspark 77d2703 [Haoyuan Li] change python api.git status 3dcace4 [Haoyuan Li] address matei's comments 91fa09d [Haoyuan Li] address patrick's comments 589eafe [Haoyuan Li] use TRY_CACHE instead of MUST_CACHE 64348b2 [Haoyuan Li] update conf docs. ed73e19 [Haoyuan Li] Merge branch 'master' of github.com:RongGu/spark-1 619a9a8 [RongGu] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore be79d77 [RongGu] find a way to clean up some unnecessay metods and classed to make the code simpler 49cc724 [Haoyuan Li] update docs with off_headp option 4572f9f [RongGu] reserving the old apply function API of StorageLevel 04301d3 [RongGu] rename StorageLevel.TACHYON to Storage.OFF_HEAP c9aeabf [RongGu] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP 76805aa [RongGu] unifies the config properties name prefix; add the configs into docs/configuration.md e700d9c [RongGu] add the SparkTachyonHdfsLR example and some comments fd84156 [RongGu] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix 939e467 [Haoyuan Li] 0.4.1-thrift from maven central 86a2eab [Haoyuan Li] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1 16c5798 [RongGu] make the dependency on tachyon as tachyon-0.4.1-thrift eacb2e8 [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 bbeb4de [RongGu] fix the JsonProtocolSuite test failure problem 6adb58f [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 d827250 [RongGu] fix JsonProtocolSuie test failure 716e93b [Haoyuan Li] revert the version ca14469 [Haoyuan Li] bump tachyon version to 0.4.1-thrift 2825a13 [RongGu] up-merging to the current master branch of the apache spark 6a22c1a [Haoyuan Li] fix scalastyle 8968b67 [Haoyuan Li] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client. 77be7e8 [RongGu] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice. 1dcadf9 [Haoyuan Li] typo bf278fa [Haoyuan Li] fix python tests e82909c [Haoyuan Li] minor cleanup 776a56c [Haoyuan Li] address patrick's and ali's comments from the previous PR 8859371 [Haoyuan Li] various minor fixes and clean up e3ddbba [Haoyuan Li] add doc to use Tachyon cache mode. fcaeab2 [Haoyuan Li] address Aaron's comment e554b1e [Haoyuan Li] add python code 47304b3 [Haoyuan Li] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels. dc8ef24 [Haoyuan Li] add old storelevel constructor e01a271 [Haoyuan Li] update tachyon 0.4.1 8011a96 [RongGu] fix a brought-in mistake in StorageLevel 70ca182 [RongGu] a bit change in comment 556978b [RongGu] fix the scalastyle errors 791189b [RongGu] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark
2014-04-04 23:36:24 -04:00
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</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
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</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
</tr>
<tr>
<td><code>spark.eventLog.overwrite</code></td>
<td>false</td>
<td>
Whether to overwrite any existing files.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.eventLog.buffer.kb</code></td>
<td>100k</td>
<td>
Buffer size to use when writing to output streams, in KiB unless otherwise specified.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</td>
</tr>
[SPARK-28869][CORE] Roll over event log files ### What changes were proposed in this pull request? This patch is a part of [SPARK-28594](https://issues.apache.org/jira/browse/SPARK-28594) and design doc for SPARK-28594 is linked here: https://docs.google.com/document/d/12bdCC4nA58uveRxpeo8k7kGOI2NRTXmXyBOweSi4YcY/edit?usp=sharing This patch proposes adding new feature to event logging, rolling event log files via configured file size. Previously event logging is done with single file and related codebase (`EventLoggingListener`/`FsHistoryProvider`) is tightly coupled with it. This patch adds layer on both reader (`EventLogFileReader`) and writer (`EventLogFileWriter`) to decouple implementation details between "handling events" and "how to read/write events from/to file". This patch adds two properties, `spark.eventLog.rollLog` and `spark.eventLog.rollLog.maxFileSize` which provides configurable behavior of rolling log. The feature is disabled by default, as we only expect huge event log for huge/long-running application. For other cases single event log file would be sufficient and still simpler. ### Why are the changes needed? This is a part of SPARK-28594 which addresses event log growing infinitely for long-running application. This patch itself also provides some option for the situation where event log file gets huge and consume their storage. End users may give up replaying their events and want to delete the event log file, but given application is still running and writing the file, it's not safe to delete the file. End users will be able to delete some of old files after applying rolling over event log. ### Does this PR introduce any user-facing change? No, as the new feature is turned off by default. ### How was this patch tested? Added unit tests, as well as basic manual tests. Basic manual tests - ran SHS, ran structured streaming query with roll event log enabled, verified split files are generated as well as SHS can load these files, with handling app status as incomplete/complete. Closes #25670 from HeartSaVioR/SPARK-28869. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-17 14:15:25 -04:00
<tr>
<td><code>spark.eventLog.rolling.enabled</code></td>
<td>false</td>
<td>
Whether rolling over event log files is enabled. If set to true, it cuts down each event
log file to the configured size.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
[SPARK-28869][CORE] Roll over event log files ### What changes were proposed in this pull request? This patch is a part of [SPARK-28594](https://issues.apache.org/jira/browse/SPARK-28594) and design doc for SPARK-28594 is linked here: https://docs.google.com/document/d/12bdCC4nA58uveRxpeo8k7kGOI2NRTXmXyBOweSi4YcY/edit?usp=sharing This patch proposes adding new feature to event logging, rolling event log files via configured file size. Previously event logging is done with single file and related codebase (`EventLoggingListener`/`FsHistoryProvider`) is tightly coupled with it. This patch adds layer on both reader (`EventLogFileReader`) and writer (`EventLogFileWriter`) to decouple implementation details between "handling events" and "how to read/write events from/to file". This patch adds two properties, `spark.eventLog.rollLog` and `spark.eventLog.rollLog.maxFileSize` which provides configurable behavior of rolling log. The feature is disabled by default, as we only expect huge event log for huge/long-running application. For other cases single event log file would be sufficient and still simpler. ### Why are the changes needed? This is a part of SPARK-28594 which addresses event log growing infinitely for long-running application. This patch itself also provides some option for the situation where event log file gets huge and consume their storage. End users may give up replaying their events and want to delete the event log file, but given application is still running and writing the file, it's not safe to delete the file. End users will be able to delete some of old files after applying rolling over event log. ### Does this PR introduce any user-facing change? No, as the new feature is turned off by default. ### How was this patch tested? Added unit tests, as well as basic manual tests. Basic manual tests - ran SHS, ran structured streaming query with roll event log enabled, verified split files are generated as well as SHS can load these files, with handling app status as incomplete/complete. Closes #25670 from HeartSaVioR/SPARK-28869. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-17 14:15:25 -04:00
</tr>
<tr>
<td><code>spark.eventLog.rolling.maxFileSize</code></td>
<td>128m</td>
<td>
When <code>spark.eventLog.rolling.enabled=true</code>, specifies the max size of event log file before it's rolled over.
[SPARK-28869][CORE] Roll over event log files ### What changes were proposed in this pull request? This patch is a part of [SPARK-28594](https://issues.apache.org/jira/browse/SPARK-28594) and design doc for SPARK-28594 is linked here: https://docs.google.com/document/d/12bdCC4nA58uveRxpeo8k7kGOI2NRTXmXyBOweSi4YcY/edit?usp=sharing This patch proposes adding new feature to event logging, rolling event log files via configured file size. Previously event logging is done with single file and related codebase (`EventLoggingListener`/`FsHistoryProvider`) is tightly coupled with it. This patch adds layer on both reader (`EventLogFileReader`) and writer (`EventLogFileWriter`) to decouple implementation details between "handling events" and "how to read/write events from/to file". This patch adds two properties, `spark.eventLog.rollLog` and `spark.eventLog.rollLog.maxFileSize` which provides configurable behavior of rolling log. The feature is disabled by default, as we only expect huge event log for huge/long-running application. For other cases single event log file would be sufficient and still simpler. ### Why are the changes needed? This is a part of SPARK-28594 which addresses event log growing infinitely for long-running application. This patch itself also provides some option for the situation where event log file gets huge and consume their storage. End users may give up replaying their events and want to delete the event log file, but given application is still running and writing the file, it's not safe to delete the file. End users will be able to delete some of old files after applying rolling over event log. ### Does this PR introduce any user-facing change? No, as the new feature is turned off by default. ### How was this patch tested? Added unit tests, as well as basic manual tests. Basic manual tests - ran SHS, ran structured streaming query with roll event log enabled, verified split files are generated as well as SHS can load these files, with handling app status as incomplete/complete. Closes #25670 from HeartSaVioR/SPARK-28869. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-17 14:15:25 -04:00
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
[SPARK-28869][CORE] Roll over event log files ### What changes were proposed in this pull request? This patch is a part of [SPARK-28594](https://issues.apache.org/jira/browse/SPARK-28594) and design doc for SPARK-28594 is linked here: https://docs.google.com/document/d/12bdCC4nA58uveRxpeo8k7kGOI2NRTXmXyBOweSi4YcY/edit?usp=sharing This patch proposes adding new feature to event logging, rolling event log files via configured file size. Previously event logging is done with single file and related codebase (`EventLoggingListener`/`FsHistoryProvider`) is tightly coupled with it. This patch adds layer on both reader (`EventLogFileReader`) and writer (`EventLogFileWriter`) to decouple implementation details between "handling events" and "how to read/write events from/to file". This patch adds two properties, `spark.eventLog.rollLog` and `spark.eventLog.rollLog.maxFileSize` which provides configurable behavior of rolling log. The feature is disabled by default, as we only expect huge event log for huge/long-running application. For other cases single event log file would be sufficient and still simpler. ### Why are the changes needed? This is a part of SPARK-28594 which addresses event log growing infinitely for long-running application. This patch itself also provides some option for the situation where event log file gets huge and consume their storage. End users may give up replaying their events and want to delete the event log file, but given application is still running and writing the file, it's not safe to delete the file. End users will be able to delete some of old files after applying rolling over event log. ### Does this PR introduce any user-facing change? No, as the new feature is turned off by default. ### How was this patch tested? Added unit tests, as well as basic manual tests. Basic manual tests - ran SHS, ran structured streaming query with roll event log enabled, verified split files are generated as well as SHS can load these files, with handling app status as incomplete/complete. Closes #25670 from HeartSaVioR/SPARK-28869. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com> Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-17 14:15:25 -04:00
</tr>
<tr>
<td><code>spark.ui.dagGraph.retainedRootRDDs</code></td>
<td>Int.MaxValue</td>
<td>
How many DAG graph nodes the Spark UI and status APIs remember before garbage collecting.
</td>
[SPARK-30911][CORE][DOC] Add version information to the configuration of Status ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Status`. 2.Update the docs of `Status`. 3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847 I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |   spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |   spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |   spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27848 from beliefer/add-version-to-status-config. Lead-authored-by: beliefer <beliefer@163.com> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 22:03:47 -04:00
<td>2.1.0</td>
</tr>
<tr>
<td><code>spark.ui.enabled</code></td>
<td>true</td>
<td>
Whether to run the web UI for the Spark application.
</td>
[SPARK-30914][CORE][DOC] Add version information to the configuration of UI ### What changes were proposed in this pull request? 1.Add version information to the configuration of `UI`. 2.Update the docs of `UI`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |   spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |   spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 | spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |   spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |   spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27806 from beliefer/add-version-to-UI-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 21:08:57 -05:00
<td>1.1.1</td>
</tr>
<tr>
<td><code>spark.ui.killEnabled</code></td>
<td>true</td>
<td>
Allows jobs and stages to be killed from the web UI.
</td>
[SPARK-30914][CORE][DOC] Add version information to the configuration of UI ### What changes were proposed in this pull request? 1.Add version information to the configuration of `UI`. 2.Update the docs of `UI`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |   spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |   spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 | spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |   spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |   spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27806 from beliefer/add-version-to-UI-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 21:08:57 -05:00
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.ui.liveUpdate.period</code></td>
<td>100ms</td>
<td>
How often to update live entities. -1 means "never update" when replaying applications,
meaning only the last write will happen. For live applications, this avoids a few
operations that we can live without when rapidly processing incoming task events.
</td>
[SPARK-30911][CORE][DOC] Add version information to the configuration of Status ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Status`. 2.Update the docs of `Status`. 3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847 I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |   spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |   spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |   spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27848 from beliefer/add-version-to-status-config. Lead-authored-by: beliefer <beliefer@163.com> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 22:03:47 -04:00
<td>2.3.0</td>
</tr>
<tr>
<td><code>spark.ui.liveUpdate.minFlushPeriod</code></td>
<td>1s</td>
<td>
Minimum time elapsed before stale UI data is flushed. This avoids UI staleness when incoming
task events are not fired frequently.
</td>
[SPARK-30911][CORE][DOC] Add version information to the configuration of Status ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Status`. 2.Update the docs of `Status`. 3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847 I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |   spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |   spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |   spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27848 from beliefer/add-version-to-status-config. Lead-authored-by: beliefer <beliefer@163.com> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 22:03:47 -04:00
<td>2.4.2</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.ui.port</code></td>
<td>4040</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
Port for your application's dashboard, which shows memory and workload data.
</td>
[SPARK-30914][CORE][DOC] Add version information to the configuration of UI ### What changes were proposed in this pull request? 1.Add version information to the configuration of `UI`. 2.Update the docs of `UI`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |   spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |   spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 | spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |   spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |   spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27806 from beliefer/add-version-to-UI-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 21:08:57 -05:00
<td>0.7.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.ui.retainedJobs</code></td>
<td>1000</td>
<td>
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.
</td>
[SPARK-30911][CORE][DOC] Add version information to the configuration of Status ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Status`. 2.Update the docs of `Status`. 3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847 I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |   spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |   spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |   spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27848 from beliefer/add-version-to-status-config. Lead-authored-by: beliefer <beliefer@163.com> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 22:03:47 -04:00
<td>1.2.0</td>
</tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.ui.retainedStages</code></td>
<td>1000</td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td>
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.
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</td>
[SPARK-30911][CORE][DOC] Add version information to the configuration of Status ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Status`. 2.Update the docs of `Status`. 3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847 I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |   spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |   spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |   spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27848 from beliefer/add-version-to-status-config. Lead-authored-by: beliefer <beliefer@163.com> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 22:03:47 -04:00
<td>0.9.0</td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</tr>
<tr>
<td><code>spark.ui.retainedTasks</code></td>
<td>100000</td>
<td>
How many tasks in one stage the Spark UI and status APIs remember before garbage collecting.
This is a target maximum, and fewer elements may be retained in some circumstances.
</td>
[SPARK-30911][CORE][DOC] Add version information to the configuration of Status ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Status`. 2.Update the docs of `Status`. 3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847 I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |   spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |   spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |   spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27848 from beliefer/add-version-to-status-config. Lead-authored-by: beliefer <beliefer@163.com> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 22:03:47 -04:00
<td>2.0.1</td>
</tr>
<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>
[SPARK-30914][CORE][DOC] Add version information to the configuration of UI ### What changes were proposed in this pull request? 1.Add version information to the configuration of `UI`. 2.Update the docs of `UI`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |   spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |   spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 | spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |   spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |   spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27806 from beliefer/add-version-to-UI-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 21:08:57 -05:00
<td>2.1.0</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>
[SPARK-30914][CORE][DOC] Add version information to the configuration of UI ### What changes were proposed in this pull request? 1.Add version information to the configuration of `UI`. 2.Update the docs of `UI`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |   spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |   spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 | spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |   spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |   spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27806 from beliefer/add-version-to-UI-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 21:08:57 -05:00
<td>2.1.0</td>
</tr>
<tr>
<td><code>spark.ui.proxyRedirectUri</code></td>
<td></td>
<td>
Where to address redirects when Spark is running behind a proxy. This will make Spark
modify redirect responses so they point to the proxy server, instead of the Spark UI's own
address. This should be only the address of the server, without any prefix paths for the
application; the prefix should be set either by the proxy server itself (by adding the
<code>X-Forwarded-Context</code> request header), or by setting the proxy base in the Spark
app's configuration.
</td>
[SPARK-30914][CORE][DOC] Add version information to the configuration of UI ### What changes were proposed in this pull request? 1.Add version information to the configuration of `UI`. 2.Update the docs of `UI`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |   spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |   spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 | spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |   spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |   spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27806 from beliefer/add-version-to-UI-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 21:08:57 -05:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.ui.showConsoleProgress</code></td>
<td>false</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.
<br/>
<em>Note:</em> In shell environment, the default value of spark.ui.showConsoleProgress is true.
</td>
[SPARK-30914][CORE][DOC] Add version information to the configuration of UI ### What changes were proposed in this pull request? 1.Add version information to the configuration of `UI`. 2.Update the docs of `UI`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |   spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |   spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 | spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |   spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |   spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27806 from beliefer/add-version-to-UI-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 21:08:57 -05:00
<td>1.2.1</td>
</tr>
<tr>
<td><code>spark.ui.custom.executor.log.url</code></td>
<td>(none)</td>
<td>
Specifies custom spark executor log URL for supporting external log service instead of using cluster
managers' application log URLs in Spark UI. Spark will support some path variables via patterns
which can vary on cluster manager. Please check the documentation for your cluster manager to
see which patterns are supported, if any. <p/>
Please note that this configuration also replaces original log urls in event log,
which will be also effective when accessing the application on history server. The new log urls must be
permanent, otherwise you might have dead link for executor log urls.
<p/>
For now, only YARN mode supports this configuration
</td>
[SPARK-30914][CORE][DOC] Add version information to the configuration of UI ### What changes were proposed in this pull request? 1.Add version information to the configuration of `UI`. 2.Update the docs of `UI`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |   spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |   spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 | spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |   spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |   spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27806 from beliefer/add-version-to-UI-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 21:08:57 -05:00
<td>3.0.0</td>
</tr>
<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>
[SPARK-30889][SPARK-30913][CORE][DOC] Add version information to the configuration of Tests.scala and Worker ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Tests` and `Worker`. 2.Update the docs of `Worker`. I sorted out some information of `Tests` show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.testing.memory | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-395d07dcd46359cca610ce74357f0bb4 |   spark.testing.dynamicAllocation.scheduleInterval | 2.3.0 | SPARK-22864 | 4e9e6aee44bb2ddb41b567d659358b22fd824222#diff-b096353602813e47074ace09a3890d56 |   spark.testing | 1.0.1 | SPARK-1606 | ce57624b8232159fe3ec6db228afc622133df591#diff-d239aee594001f8391676e1047a0381e |   spark.test.noStageRetry | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.testing.reservedMemory | 1.6.0 | SPARK-12081 | 84c44b500b5c90dffbe1a6b0aa86f01699b09b96#diff-395d07dcd46359cca610ce74357f0bb4 | spark.testing.nHosts | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.testing.nExecutorsPerHost | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.testing.nCoresPerExecutor | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.resources.warnings.testing | 3.1.0 | SPARK-29148 | 496f6ac86001d284cbfb7488a63dd3a168919c0f#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.testing.resourceProfileManager | 3.1.0 | SPARK-29148 | 496f6ac86001d284cbfb7488a63dd3a168919c0f#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   I sorted out some information of `Worker` show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.worker.resourcesFile | 3.0.0 | SPARK-27369 | 7cbe01e8efc3f6cd3a0cac4bcfadea8fcc74a955#diff-b2fc8d6ab7ac5735085e2d6cfacb95da |   spark.worker.timeout | 0.6.2 | None | e395aa295aeec6767df798bf1002b1f30983c1cd#diff-776a630ac2b2ec5fe85c07ca20a58fc0 |   spark.worker.driverTerminateTimeout | 2.1.2 | SPARK-20843 | ebd72f453aa0b4f68760d28b3e93e6dd33856659#diff-829a8674171f92acd61007bedb1bfa4f |   spark.worker.cleanup.enabled | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.cleanup.interval | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.cleanup.appDataTtl | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.preferConfiguredMasterAddress | 2.2.1 | SPARK-20529 | 75e5ea294c15ecfb7366ae15dce196aa92c87ca4#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-48ca297b6536cb92362bec1487581f05 |   spark.worker.ui.retainedExecutors | 1.5.0 | SPARK-9202 | c0686668ae6a92b6bb4801a55c3b78aedbee816a#diff-916ca56b663f178f302c265b7ef38499 | spark.worker.ui.retainedDrivers | 1.5.0 | SPARK-9202 | c0686668ae6a92b6bb4801a55c3b78aedbee816a#diff-916ca56b663f178f302c265b7ef38499 | spark.worker.ui.compressedLogFileLengthCacheSize | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-d239aee594001f8391676e1047a0381e |   spark.worker.decommission.enabled | 3.1.0 | SPARK-20628 | d273a2bb0fac452a97f5670edd69d3e452e3e57e#diff-b2fc8d6ab7ac5735085e2d6cfacb95da |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27783 from beliefer/add-version-to-tests-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-04 21:58:21 -05:00
<td>1.5.0</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>
[SPARK-30889][SPARK-30913][CORE][DOC] Add version information to the configuration of Tests.scala and Worker ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Tests` and `Worker`. 2.Update the docs of `Worker`. I sorted out some information of `Tests` show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.testing.memory | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-395d07dcd46359cca610ce74357f0bb4 |   spark.testing.dynamicAllocation.scheduleInterval | 2.3.0 | SPARK-22864 | 4e9e6aee44bb2ddb41b567d659358b22fd824222#diff-b096353602813e47074ace09a3890d56 |   spark.testing | 1.0.1 | SPARK-1606 | ce57624b8232159fe3ec6db228afc622133df591#diff-d239aee594001f8391676e1047a0381e |   spark.test.noStageRetry | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.testing.reservedMemory | 1.6.0 | SPARK-12081 | 84c44b500b5c90dffbe1a6b0aa86f01699b09b96#diff-395d07dcd46359cca610ce74357f0bb4 | spark.testing.nHosts | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.testing.nExecutorsPerHost | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.testing.nCoresPerExecutor | 3.0.0 | SPARK-26491 | 1a641525e60039cc6b10816e946cb6f44b3e2696#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.resources.warnings.testing | 3.1.0 | SPARK-29148 | 496f6ac86001d284cbfb7488a63dd3a168919c0f#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   spark.testing.resourceProfileManager | 3.1.0 | SPARK-29148 | 496f6ac86001d284cbfb7488a63dd3a168919c0f#diff-8b4ea8f3b0cc1e7ce7e943de1abbb165 |   I sorted out some information of `Worker` show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.worker.resourcesFile | 3.0.0 | SPARK-27369 | 7cbe01e8efc3f6cd3a0cac4bcfadea8fcc74a955#diff-b2fc8d6ab7ac5735085e2d6cfacb95da |   spark.worker.timeout | 0.6.2 | None | e395aa295aeec6767df798bf1002b1f30983c1cd#diff-776a630ac2b2ec5fe85c07ca20a58fc0 |   spark.worker.driverTerminateTimeout | 2.1.2 | SPARK-20843 | ebd72f453aa0b4f68760d28b3e93e6dd33856659#diff-829a8674171f92acd61007bedb1bfa4f |   spark.worker.cleanup.enabled | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.cleanup.interval | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.cleanup.appDataTtl | 1.0.0 | SPARK-1154 | 1440154c27ca48b5a75103eccc9057286d3f6ca8#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.preferConfiguredMasterAddress | 2.2.1 | SPARK-20529 | 75e5ea294c15ecfb7366ae15dce196aa92c87ca4#diff-916ca56b663f178f302c265b7ef38499 |   spark.worker.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-48ca297b6536cb92362bec1487581f05 |   spark.worker.ui.retainedExecutors | 1.5.0 | SPARK-9202 | c0686668ae6a92b6bb4801a55c3b78aedbee816a#diff-916ca56b663f178f302c265b7ef38499 | spark.worker.ui.retainedDrivers | 1.5.0 | SPARK-9202 | c0686668ae6a92b6bb4801a55c3b78aedbee816a#diff-916ca56b663f178f302c265b7ef38499 | spark.worker.ui.compressedLogFileLengthCacheSize | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-d239aee594001f8391676e1047a0381e |   spark.worker.decommission.enabled | 3.1.0 | SPARK-20628 | d273a2bb0fac452a97f5670edd69d3e452e3e57e#diff-b2fc8d6ab7ac5735085e2d6cfacb95da |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27783 from beliefer/add-version-to-tests-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-04 21:58:21 -05:00
<td>1.5.0</td>
</tr>
<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>
[SPARK-31215][SQL][DOC] Add version information to the static configuration of SQL ### What changes were proposed in this pull request? Add version information to the static configuration of `SQL`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.sql.warehouse.dir | 2.0.0 | SPARK-14994 | 054f991c4350af1350af7a4109ee77f4a34822f0#diff-32bb9518401c0948c5ea19377b5069ab |   spark.sql.catalogImplementation | 2.0.0 | SPARK-14720 and SPARK-13643 | 8fc267ab3322e46db81e725a5cb1adb5a71b2b4d#diff-6bdad48cfc34314e89599655442ff210 |   spark.sql.globalTempDatabase | 2.1.0 | SPARK-17338 | 23ddff4b2b2744c3dc84d928e144c541ad5df376#diff-6bdad48cfc34314e89599655442ff210 |   spark.sql.sources.schemaStringLengthThreshold | 1.3.1 | SPARK-6024 | 6200f0709c5c8440decae8bf700d7859f32ac9d5#diff-41ef65b9ef5b518f77e2a03559893f4d | 1.3 spark.sql.filesourceTableRelationCacheSize | 2.2.0 | SPARK-19265 | 9d9d67c7957f7cbbdbe889bdbc073568b2bfbb16#diff-32bb9518401c0948c5ea19377b5069ab | spark.sql.codegen.cache.maxEntries | 2.4.0 | SPARK-24727 | b2deef64f604ddd9502a31105ed47cb63470ec85#diff-5081b9388de3add800b6e4a6ddf55c01 | spark.sql.codegen.comments | 2.0.0 | SPARK-15680 | f0e8738c1ec0e4c5526aeada6f50cf76428f9afd#diff-8bcc5aea39c73d4bf38aef6f6951d42c |   spark.sql.debug | 2.1.0 | SPARK-17899 | db8784feaa605adcbd37af4bc8b7146479b631f8#diff-32bb9518401c0948c5ea19377b5069ab |   spark.sql.hive.thriftServer.singleSession | 1.6.0 | SPARK-11089 | 167ea61a6a604fd9c0b00122a94d1bc4b1de24ff#diff-ff50aea397a607b79df9bec6f2a841db |   spark.sql.extensions | 2.2.0 | SPARK-18127 | f0de600797ff4883927d0c70732675fd8629e239#diff-5081b9388de3add800b6e4a6ddf55c01 |   spark.sql.queryExecutionListeners | 2.3.0 | SPARK-19558 | bd4eb9ce57da7bacff69d9ed958c94f349b7e6fb#diff-5081b9388de3add800b6e4a6ddf55c01 |   spark.sql.streaming.streamingQueryListeners | 2.4.0 | SPARK-24479 | 7703b46d2843db99e28110c4c7ccf60934412504#diff-5081b9388de3add800b6e4a6ddf55c01 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.sql.broadcastExchange.maxThreadThreshold | 3.0.0 | SPARK-26601 | 126310ca68f2f248ea8b312c4637eccaba2fdc2b#diff-5081b9388de3add800b6e4a6ddf55c01 |   spark.sql.subquery.maxThreadThreshold | 2.4.6 | SPARK-30556 | 2fc562cafd71ec8f438f37a28b65118906ab2ad2#diff-5081b9388de3add800b6e4a6ddf55c01 |   spark.sql.event.truncate.length | 3.0.0 | SPARK-27045 | e60d8fce0b0cf2a6d766ea2fc5f994546550570a#diff-5081b9388de3add800b6e4a6ddf55c01 | spark.sql.legacy.sessionInitWithConfigDefaults | 3.0.0 | SPARK-27253 | 83f628b57da39ad9732d1393aebac373634a2eb9#diff-5081b9388de3add800b6e4a6ddf55c01 | spark.sql.defaultUrlStreamHandlerFactory.enabled | 3.0.0 | SPARK-25694 | 8469614c0513fbed87977d4e741649db3fdd8add#diff-5081b9388de3add800b6e4a6ddf55c01 | spark.sql.streaming.ui.enabled | 3.0.0 | SPARK-29543 | f9b86370cb04b72a4f00cbd4d60873960aa2792c#diff-5081b9388de3add800b6e4a6ddf55c01 |   spark.sql.streaming.ui.retainedProgressUpdates | 3.0.0 | SPARK-29543 | f9b86370cb04b72a4f00cbd4d60873960aa2792c#diff-5081b9388de3add800b6e4a6ddf55c01 |   spark.sql.streaming.ui.retainedQueries | 3.0.0 | SPARK-29543 | f9b86370cb04b72a4f00cbd4d60873960aa2792c#diff-5081b9388de3add800b6e4a6ddf55c01 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27981 from beliefer/add-version-to-sql-static-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:31:25 -04:00
<td>1.5.0</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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.0.0</td>
</tr>
<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>
[SPARK-30911][CORE][DOC] Add version information to the configuration of Status ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Status`. 2.Update the docs of `Status`. 3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847 I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |   spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |   spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |   spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27848 from beliefer/add-version-to-status-config. Lead-authored-by: beliefer <beliefer@163.com> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 22:03:47 -04:00
<td>2.0.0</td>
</tr>
<tr>
<td><code>spark.ui.filters</code></td>
<td>None</td>
<td>
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>.
<br />Filter parameters can also be specified in the configuration, by setting config entries
of the form <code>spark.&lt;class name of filter&gt;.param.&lt;param name&gt;=&lt;value&gt;</code>
<br />For example:
<br /><code>spark.ui.filters=com.test.filter1</code>
<br /><code>spark.com.test.filter1.param.name1=foo</code>
<br /><code>spark.com.test.filter1.param.name2=bar</code>
</td>
[SPARK-30914][CORE][DOC] Add version information to the configuration of UI ### What changes were proposed in this pull request? 1.Add version information to the configuration of `UI`. 2.Update the docs of `UI`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |   spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |   spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 | spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |   spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |   spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27806 from beliefer/add-version-to-UI-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 21:08:57 -05:00
<td>1.0.0</td>
</tr>
[SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP requestHeaderSize ## What changes were proposed in this pull request? Introducing spark.ui.requestHeaderSize for configuring Jetty's HTTP requestHeaderSize. This way long authorization field does not lead to HTTP 413. ## How was this patch tested? Manually with curl (which version must be at least 7.55). With the original default value (8k limit): ```bash # Starting history server with default requestHeaderSize $ ./sbin/start-history-server.sh starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out # Creating huge header $ echo -n "X-Custom-Header: " > cookie $ printf 'A%.0s' {1..9500} >> cookie # HTTP GET with huge header fails with 431 $ curl -H cookie http://458apiros-MBP.lan:18080/ <h1>Bad Message 431</h1><pre>reason: Request Header Fields Too Large</pre> # The log contains the error $ tail -1 /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out 18/11/19 21:24:28 WARN HttpParser: Header is too large 8193>8192 ``` After: ```bash # Creating the history properties file with the increased requestHeaderSize $ echo spark.ui.requestHeaderSize=10000 > history.properties # Starting Spark History Server with the settings $ ./sbin/start-history-server.sh --properties-file history.properties starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out # HTTP GET with huge header gives back HTML5 (I have added here only just a part of the response) $ curl -H cookie http://458apiros-MBP.lan:18080/ <!DOCTYPE html><html> <head>... <link rel="shortcut icon" href="/static/spark-logo-77x50px-hd.png"></link> <title>History Server</title> </head> <body> ... ``` Closes #23090 from attilapiros/JettyHeaderSize. Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com> Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-11-20 09:56:22 -05:00
<tr>
<td><code>spark.ui.requestHeaderSize</code></td>
<td>8k</td>
<td>
The maximum allowed size for a HTTP request header, in bytes unless otherwise specified.
This setting applies for the Spark History Server too.
</td>
[SPARK-30914][CORE][DOC] Add version information to the configuration of UI ### What changes were proposed in this pull request? 1.Add version information to the configuration of `UI`. 2.Update the docs of `UI`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.ui.showConsoleProgress | 1.2.1 | SPARK-4017 | 04b1bdbae31c3039125100e703121daf7d9dabf5#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.consoleProgress.update.interval | 2.1.0 | SPARK-16919 | e076fb05ac83a3ed6995e29bb03ea07ea05e39db#diff-fbf4e388a66b6a37e984b91cd71a3e2c |   spark.ui.enabled | 1.1.1 | SPARK-3490 | 937de93e80e6d299c4d08be426da2d5bc2d66f98#diff-364713d7776956cb8b0a771e9b62f82d |   spark.ui.port | 0.7.0 | None | f03d9760fd8ac67fd0865cb355ba75d2eff507fe#diff-ed8dbcebe16fda5ecd6df1a981dc6fee |   spark.ui.filters | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.allowFramingFrom | 1.6.0 | SPARK-10589 | 5dbaf3d3911bbfa003bc75459aaad66b4f6e0c67#diff-f79a5ead735b3d0b34b6b94486918e1c |   spark.ui.reverseProxy | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.reverseProxyUrl | 2.1.0 | SPARK-15487 | 92ce8d4849a0341c4636e70821b7be57ad3055b1#diff-364713d7776956cb8b0a771e9b62f82d | spark.ui.killEnabled | 1.0.0 | SPARK-1202 | 211f97447b5f078afcb1619a08d2e2349325f61a#diff-a40023c80383451b6e29ee7a6e0593e9 | spark.ui.threadDumpsEnabled | 1.2.0 | SPARK-611 | 866c7bbe56f9c7fd96d3f4afe8a76405dc877a6e#diff-5d18fb70c572369a0fff0b97de94f265 |   spark.ui.prometheus.enabled | 3.0.0 | SPARK-29064 | bbfaadb280a80b511a98d18881641c6d9851dd51#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.xXssProtection | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.xContentTypeOptions.enabled | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.strictTransportSecurity | 2.3.0 | SPARK-22188 | 5a07aca4d464e96d75ea17bf6768e24b829872ec#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.requestHeaderSize | 2.2.3 | SPARK-26118 | 9ceee6f188e6c3794d31ce15cc61d29f907bebf7#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.timeline.tasks.maximum | 1.4.0 | SPARK-7296 | a5f7b3b9c7f05598a1cc8e582e5facee1029cd5e#diff-fa4cfb2cce1b925f55f41f2dfa8c8501 |   spark.acls.enable | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.view.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.admin.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls | 1.1.0 | SPARK-1890 and SPARK-1891 | e3fe6571decfdc406ec6d505fd92f9f2b85a618c#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.modify.acls.groups | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.user.groups.mapping | 2.0.0 | SPARK-4224 | ae79032dcf160796851ca29116cca146c4d86ada#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.ui.proxyRedirectUri | 3.0.0 | SPARK-30240 | a9fbd310300e57ed58818d7347f3c3172701c491#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   spark.ui.custom.executor.log.url | 3.0.0 | SPARK-26792 | d5bda2c9e8dde6afc075cc7f65b15fa9aa82231c#diff-f70174ad0759db1fb4cb36a7ff9324a7 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27806 from beliefer/add-version-to-UI-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-05 21:08:57 -05:00
<td>2.2.3</td>
[SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP requestHeaderSize ## What changes were proposed in this pull request? Introducing spark.ui.requestHeaderSize for configuring Jetty's HTTP requestHeaderSize. This way long authorization field does not lead to HTTP 413. ## How was this patch tested? Manually with curl (which version must be at least 7.55). With the original default value (8k limit): ```bash # Starting history server with default requestHeaderSize $ ./sbin/start-history-server.sh starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out # Creating huge header $ echo -n "X-Custom-Header: " > cookie $ printf 'A%.0s' {1..9500} >> cookie # HTTP GET with huge header fails with 431 $ curl -H cookie http://458apiros-MBP.lan:18080/ <h1>Bad Message 431</h1><pre>reason: Request Header Fields Too Large</pre> # The log contains the error $ tail -1 /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out 18/11/19 21:24:28 WARN HttpParser: Header is too large 8193>8192 ``` After: ```bash # Creating the history properties file with the increased requestHeaderSize $ echo spark.ui.requestHeaderSize=10000 > history.properties # Starting Spark History Server with the settings $ ./sbin/start-history-server.sh --properties-file history.properties starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out # HTTP GET with huge header gives back HTML5 (I have added here only just a part of the response) $ curl -H cookie http://458apiros-MBP.lan:18080/ <!DOCTYPE html><html> <head>... <link rel="shortcut icon" href="/static/spark-logo-77x50px-hd.png"></link> <title>History Server</title> </head> <body> ... ``` Closes #23090 from attilapiros/JettyHeaderSize. Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com> Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-11-20 09:56:22 -05:00
</tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</table>
### Compression and Serialization
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<table class="table">
[SPARK-30908][CORE][DOC] Add version information to the configuration of Kryo ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Kryo`. 2.Update the docs of `Kryo`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.kryo.registrationRequired | 1.1.0 | SPARK-2102 | efdaeb111917dd0314f1d00ee8524bed1e2e21ca#diff-1f81c62dad0e2dfc387a974bb08c497c |   spark.kryo.registrator | 0.5.0 | None | 91c07a33d90ab0357e8713507134ecef5c14e28a#diff-792ed56b3398163fa14e8578549d0d98 | This is not a release version, do we need to record it? spark.kryo.classesToRegister | 1.2.0 | SPARK-1813 | 6bb56faea8d238ea22c2de33db93b1b39f492b3a#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.kryo.unsafe | 2.1.0 | SPARK-928 | bc167a2a53f5a795d089e8a884569b1b3e2cd439#diff-1f81c62dad0e2dfc387a974bb08c497c |   spark.kryo.pool | 3.0.0 | SPARK-26466 | 38f030725c561979ca98b2a6cc7ca6c02a1f80ed#diff-a3c6b992784f9abeb9f3047d3dcf3ed9 |   spark.kryo.referenceTracking | 0.8.0 | None | 0a8cc309211c62f8824d76618705c817edcf2424#diff-1f81c62dad0e2dfc387a974bb08c497c |   spark.kryoserializer.buffer | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-1f81c62dad0e2dfc387a974bb08c497c |   spark.kryoserializer.buffer.max | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-1f81c62dad0e2dfc387a974bb08c497c |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27734 from beliefer/add-version-to-kryo-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-02 01:14:47 -05:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td><code>spark.broadcast.compress</code></td>
<td>true</td>
<td>
Whether to compress broadcast variables before sending them. Generally a good idea.
Compression will use <code>spark.io.compression.codec</code>.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.6.0</td>
</tr>
<tr>
<td><code>spark.checkpoint.compress</code></td>
<td>false</td>
<td>
Whether to compress RDD checkpoints. Generally a good idea.
Compression will use <code>spark.io.compression.codec</code>.
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
</td>
<td>2.2.0</td>
</tr>
<tr>
<td><code>spark.io.compression.codec</code></td>
<td>lz4</td>
<td>
The codec used to compress internal data such as RDD partitions, event log, broadcast variables
and shuffle outputs. By default, Spark provides four codecs: <code>lz4</code>, <code>lzf</code>,
<code>snappy</code>, and <code>zstd</code>. You can also use fully qualified class names to specify the codec,
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
e.g.
<code>org.apache.spark.io.LZ4CompressionCodec</code>,
<code>org.apache.spark.io.LZFCompressionCodec</code>,
<code>org.apache.spark.io.SnappyCompressionCodec</code>,
and <code>org.apache.spark.io.ZStdCompressionCodec</code>.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.8.0</td>
</tr>
<tr>
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
2015-04-28 15:18:55 -04:00
<td><code>spark.io.compression.lz4.blockSize</code></td>
<td>32k</td>
<td>
Block size used in LZ4 compression, in the case when LZ4 compression codec
is used. Lowering this block size will also lower shuffle memory usage when LZ4 is used.
Default unit is bytes, unless otherwise specified.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.4.0</td>
</tr>
<tr>
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
2015-04-28 15:18:55 -04:00
<td><code>spark.io.compression.snappy.blockSize</code></td>
<td>32k</td>
<td>
Block size in Snappy compression, in the case when Snappy compression codec is used.
Lowering this block size will also lower shuffle memory usage when Snappy is used.
Default unit is bytes, unless otherwise specified.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.4.0</td>
</tr>
<tr>
<td><code>spark.io.compression.zstd.level</code></td>
<td>1</td>
<td>
Compression level for Zstd compression codec. Increasing the compression level will result in better
compression at the expense of more CPU and memory.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>2.3.0</td>
</tr>
<tr>
<td><code>spark.io.compression.zstd.bufferSize</code></td>
<td>32k</td>
<td>
Buffer size in bytes used in Zstd compression, in the case when Zstd compression codec
is used. Lowering this size will lower the shuffle memory usage when Zstd is used, but it
might increase the compression cost because of excessive JNI call overhead.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>2.3.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.kryo.classesToRegister</code></td>
<td>(none)</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.2.0</td>
</tr>
<tr>
<td><code>spark.kryo.referenceTracking</code></td>
<td>true</td>
<td>
Whether to track references to the same object when serializing data with Kryo, which is
necessary if your object graphs have loops and useful for efficiency if they contain multiple
copies of the same object. Can be disabled to improve performance if you know this is not the
case.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.8.0</td>
</tr>
<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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.1.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.kryo.registrator</code></td>
<td>(none)</td>
<td>
If you use Kryo serialization, give a comma-separated list of classes that register your custom classes with Kryo. This
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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
set to classes that extend
<a href="api/scala/org/apache/spark/serializer/KryoRegistrator.html">
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<code>KryoRegistrator</code></a>.
See the <a href="tuning.html#data-serialization">tuning guide</a> for more details.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.5.0</td>
</tr>
[SPARK-928][CORE] Add support for Unsafe-based serializer in Kryo ## What changes were proposed in this pull request? Now since we have migrated to Kryo-3.0.0 in https://issues.apache.org/jira/browse/SPARK-11416, we can gives users option to use unsafe SerDer. It can turned by setting `spark.kryo.useUnsafe` to `true` ## How was this patch tested? Ran existing tests ``` Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ basicTypes: Int unsafe:true 160 / 178 98.5 10.1 1.0X basicTypes: Long unsafe:true 210 / 218 74.9 13.4 0.8X basicTypes: Float unsafe:true 203 / 213 77.5 12.9 0.8X basicTypes: Double unsafe:true 226 / 235 69.5 14.4 0.7X Array: Int unsafe:true 1087 / 1101 14.5 69.1 0.1X Array: Long unsafe:true 2758 / 2844 5.7 175.4 0.1X Array: Float unsafe:true 1511 / 1552 10.4 96.1 0.1X Array: Double unsafe:true 2942 / 2972 5.3 187.0 0.1X Map of string->Double unsafe:true 2645 / 2739 5.9 168.2 0.1X basicTypes: Int unsafe:false 211 / 218 74.7 13.4 0.8X basicTypes: Long unsafe:false 247 / 253 63.6 15.7 0.6X basicTypes: Float unsafe:false 211 / 216 74.5 13.4 0.8X basicTypes: Double unsafe:false 227 / 233 69.2 14.4 0.7X Array: Int unsafe:false 3012 / 3032 5.2 191.5 0.1X Array: Long unsafe:false 4463 / 4515 3.5 283.8 0.0X Array: Float unsafe:false 2788 / 2868 5.6 177.2 0.1X Array: Double unsafe:false 3558 / 3752 4.4 226.2 0.0X Map of string->Double unsafe:false 2806 / 2933 5.6 178.4 0.1X ``` Author: Sandeep Singh <sandeep@techaddict.me> Author: Sandeep Singh <sandeep@origamilogic.com> Closes #12913 from techaddict/SPARK-928.
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>2.1.0</td>
[SPARK-928][CORE] Add support for Unsafe-based serializer in Kryo ## What changes were proposed in this pull request? Now since we have migrated to Kryo-3.0.0 in https://issues.apache.org/jira/browse/SPARK-11416, we can gives users option to use unsafe SerDer. It can turned by setting `spark.kryo.useUnsafe` to `true` ## How was this patch tested? Ran existing tests ``` Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ basicTypes: Int unsafe:true 160 / 178 98.5 10.1 1.0X basicTypes: Long unsafe:true 210 / 218 74.9 13.4 0.8X basicTypes: Float unsafe:true 203 / 213 77.5 12.9 0.8X basicTypes: Double unsafe:true 226 / 235 69.5 14.4 0.7X Array: Int unsafe:true 1087 / 1101 14.5 69.1 0.1X Array: Long unsafe:true 2758 / 2844 5.7 175.4 0.1X Array: Float unsafe:true 1511 / 1552 10.4 96.1 0.1X Array: Double unsafe:true 2942 / 2972 5.3 187.0 0.1X Map of string->Double unsafe:true 2645 / 2739 5.9 168.2 0.1X basicTypes: Int unsafe:false 211 / 218 74.7 13.4 0.8X basicTypes: Long unsafe:false 247 / 253 63.6 15.7 0.6X basicTypes: Float unsafe:false 211 / 216 74.5 13.4 0.8X basicTypes: Double unsafe:false 227 / 233 69.2 14.4 0.7X Array: Int unsafe:false 3012 / 3032 5.2 191.5 0.1X Array: Long unsafe:false 4463 / 4515 3.5 283.8 0.0X Array: Float unsafe:false 2788 / 2868 5.6 177.2 0.1X Array: Double unsafe:false 3558 / 3752 4.4 226.2 0.0X Map of string->Double unsafe:false 2806 / 2933 5.6 178.4 0.1X ``` Author: Sandeep Singh <sandeep@techaddict.me> Author: Sandeep Singh <sandeep@origamilogic.com> Closes #12913 from techaddict/SPARK-928.
2016-10-22 15:03:37 -04:00
</tr>
<tr>
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
2015-04-28 15:18:55 -04:00
<td><code>spark.kryoserializer.buffer.max</code></td>
<td>64m</td>
<td>
Maximum allowable size of Kryo serialization buffer, in MiB unless otherwise specified.
This must be larger than any object you attempt to serialize and must be less than 2048m.
Increase this if you get a "buffer limit exceeded" exception inside Kryo.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.4.0</td>
2012-09-27 02:22:15 -04:00
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<tr>
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
2015-04-28 15:18:55 -04:00
<td><code>spark.kryoserializer.buffer</code></td>
<td>64k</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td>
Initial size of Kryo's serialization buffer, in KiB unless otherwise specified.
Note that there will be one buffer <i>per core</i> on each worker. This buffer will grow up to
<code>spark.kryoserializer.buffer.max</code> if needed.
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.4.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<tr>
<td><code>spark.rdd.compress</code></td>
<td>false</td>
<td>
Whether to compress serialized RDD partitions (e.g. for
<code>StorageLevel.MEMORY_ONLY_SER</code> in Java
and Scala or <code>StorageLevel.MEMORY_ONLY</code> in Python).
Can save substantial space at the cost of some extra CPU time.
Compression will use <code>spark.io.compression.codec</code>.
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.6.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<tr>
<td><code>spark.serializer</code></td>
<td>
org.apache.spark.serializer.<br />JavaSerializer
</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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/org/apache/spark/serializer/Serializer.html">
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<code>org.apache.spark.Serializer</code></a>.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.5.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.0.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</table>
### Memory Management
<table class="table">
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td><code>spark.memory.fraction</code></td>
<td>0.6</td>
<td>
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
aside memory for internal metadata, user data structures, and imprecise size estimation
in the case of sparse, unusually large records. Leaving this at the default value is
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>.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.6.0</td>
</tr>
<tr>
<td><code>spark.memory.storageFraction</code></td>
<td>0.5</td>
<td>
Amount of storage memory immune to eviction, expressed as a fraction of the size of the
region set aside by <code>spark.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>.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.6.0</td>
</tr>
<tr>
<td><code>spark.memory.offHeap.enabled</code></td>
<td>false</td>
<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>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.6.0</td>
</tr>
<tr>
<td><code>spark.memory.offHeap.size</code></td>
<td>0</td>
<td>
The absolute amount of memory which can be used for off-heap allocation, in bytes unless otherwise specified.
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>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.6.0</td>
</tr>
<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>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>2.2.0</td>
</tr>
<tr>
<td><code>spark.cleaner.periodicGC.interval</code></td>
<td>30min</td>
<td>
Controls how often to trigger a garbage collection.<br><br>
This context cleaner triggers cleanups only when weak references are garbage collected.
In long-running applications with large driver JVMs, where there is little memory pressure
on the driver, this may happen very occasionally or not at all. Not cleaning at all may
lead to executors running out of disk space after a while.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.6.0</td>
</tr>
<tr>
<td><code>spark.cleaner.referenceTracking</code></td>
<td>true</td>
<td>
Enables or disables context cleaning.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.cleaner.referenceTracking.blocking</code></td>
<td>true</td>
<td>
Controls whether the cleaning thread should block on cleanup tasks (other than shuffle, which is controlled by
<code>spark.cleaner.referenceTracking.blocking.shuffle</code> Spark property).
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.cleaner.referenceTracking.blocking.shuffle</code></td>
<td>false</td>
<td>
Controls whether the cleaning thread should block on shuffle cleanup tasks.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.1.1</td>
</tr>
<tr>
<td><code>spark.cleaner.referenceTracking.cleanCheckpoints</code></td>
<td>false</td>
<td>
Controls whether to clean checkpoint files if the reference is out of scope.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.4.0</td>
</tr>
</table>
### Execution Behavior
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<table class="table">
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<tr>
<td><code>spark.broadcast.blockSize</code></td>
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
2015-04-28 15:18:55 -04:00
<td>4m</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td>
Size of each piece of a block for <code>TorrentBroadcastFactory</code>, in KiB unless otherwise
specified. 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.
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.5.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<tr>
<td><code>spark.broadcast.checksum</code></td>
<td>true</td>
<td>
Whether to enable checksum for broadcast. If enabled, broadcasts will include a checksum, which can
help detect corrupted blocks, at the cost of computing and sending a little more data. It's possible
to disable it if the network has other mechanisms to guarantee data won't be corrupted during broadcast.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>2.1.1</td>
</tr>
SPARK-1706: Allow multiple executors per worker in Standalone mode resubmit of https://github.com/apache/spark/pull/636 for a totally different algorithm https://issues.apache.org/jira/browse/SPARK-1706 In current implementation, the user has to start multiple workers in a server for starting multiple executors in a server, which introduces additional overhead due to the more JVM processes... In this patch, I changed the scheduling logic in master to enable the user to start multiple executor processes within the same JVM process. 1. user configure spark.executor.maxCoreNumPerExecutor to suggest the maximum core he/she would like to allocate to each executor 2. Master assigns the executors to the workers with the major consideration on the memoryPerExecutor and the worker.freeMemory, and tries to allocate as many as possible cores to the executor ```min(min(memoryPerExecutor, worker.freeCore), maxLeftCoreToAssign)``` where ```maxLeftCoreToAssign = maxExecutorCanAssign * maxCoreNumPerExecutor``` --------------------------------------- Other small changes include change memoryPerSlave in ApplicationDescription to memoryPerExecutor, as "Slave" is overrided to represent both worker and executor in the documents... (we have some discussion on this before?) Author: CodingCat <zhunansjtu@gmail.com> Closes #731 from CodingCat/SPARK-1706-2 and squashes the following commits: 6dee808 [CodingCat] change filter predicate fbeb7e5 [CodingCat] address the comments 940cb42 [CodingCat] avoid unnecessary allocation b8ca561 [CodingCat] revert a change 45967b4 [CodingCat] remove unused method 2eeff77 [CodingCat] stylistic fixes 12a1b32 [CodingCat] change the semantic of coresPerExecutor to exact core number f035423 [CodingCat] stylistic fix d9c1685 [CodingCat] remove unused var f595bd6 [CodingCat] recover some unintentional changes 63b3df9 [CodingCat] change the description of the parameter in the submit script 4cf61f1 [CodingCat] improve the code and docs ff011e2 [CodingCat] start multiple executors on the worker by rewriting startExeuctor logic 2c2bcc5 [CodingCat] fix wrong usage info 497ec2c [CodingCat] address andrew's comments 878402c [CodingCat] change the launching executor code f64a28d [CodingCat] typo fix 387f4ec [CodingCat] bug fix 35c462c [CodingCat] address Andrew's comments 0b64fea [CodingCat] fix compilation issue 19d3da7 [CodingCat] address the comments 5b81466 [CodingCat] remove outdated comments ec7d421 [CodingCat] test commit e5efabb [CodingCat] more java docs and consolidate canUse function a26096d [CodingCat] stylistic fix a5d629a [CodingCat] java doc b34ec0c [CodingCat] make master support multiple executors per worker
2015-04-14 16:32:06 -04:00
<tr>
<td><code>spark.executor.cores</code></td>
<td>
1 in YARN mode, all the available cores on the worker in
standalone and Mesos coarse-grained modes.
</td>
<td>
The number of cores to use on each executor.
In standalone and Mesos coarse-grained modes, for more detail, see
<a href="spark-standalone.html#Executors Scheduling">this description</a>.
SPARK-1706: Allow multiple executors per worker in Standalone mode resubmit of https://github.com/apache/spark/pull/636 for a totally different algorithm https://issues.apache.org/jira/browse/SPARK-1706 In current implementation, the user has to start multiple workers in a server for starting multiple executors in a server, which introduces additional overhead due to the more JVM processes... In this patch, I changed the scheduling logic in master to enable the user to start multiple executor processes within the same JVM process. 1. user configure spark.executor.maxCoreNumPerExecutor to suggest the maximum core he/she would like to allocate to each executor 2. Master assigns the executors to the workers with the major consideration on the memoryPerExecutor and the worker.freeMemory, and tries to allocate as many as possible cores to the executor ```min(min(memoryPerExecutor, worker.freeCore), maxLeftCoreToAssign)``` where ```maxLeftCoreToAssign = maxExecutorCanAssign * maxCoreNumPerExecutor``` --------------------------------------- Other small changes include change memoryPerSlave in ApplicationDescription to memoryPerExecutor, as "Slave" is overrided to represent both worker and executor in the documents... (we have some discussion on this before?) Author: CodingCat <zhunansjtu@gmail.com> Closes #731 from CodingCat/SPARK-1706-2 and squashes the following commits: 6dee808 [CodingCat] change filter predicate fbeb7e5 [CodingCat] address the comments 940cb42 [CodingCat] avoid unnecessary allocation b8ca561 [CodingCat] revert a change 45967b4 [CodingCat] remove unused method 2eeff77 [CodingCat] stylistic fixes 12a1b32 [CodingCat] change the semantic of coresPerExecutor to exact core number f035423 [CodingCat] stylistic fix d9c1685 [CodingCat] remove unused var f595bd6 [CodingCat] recover some unintentional changes 63b3df9 [CodingCat] change the description of the parameter in the submit script 4cf61f1 [CodingCat] improve the code and docs ff011e2 [CodingCat] start multiple executors on the worker by rewriting startExeuctor logic 2c2bcc5 [CodingCat] fix wrong usage info 497ec2c [CodingCat] address andrew's comments 878402c [CodingCat] change the launching executor code f64a28d [CodingCat] typo fix 387f4ec [CodingCat] bug fix 35c462c [CodingCat] address Andrew's comments 0b64fea [CodingCat] fix compilation issue 19d3da7 [CodingCat] address the comments 5b81466 [CodingCat] remove outdated comments ec7d421 [CodingCat] test commit e5efabb [CodingCat] more java docs and consolidate canUse function a26096d [CodingCat] stylistic fix a5d629a [CodingCat] java doc b34ec0c [CodingCat] make master support multiple executors per worker
2015-04-14 16:32:06 -04:00
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>1.0.0</td>
SPARK-1706: Allow multiple executors per worker in Standalone mode resubmit of https://github.com/apache/spark/pull/636 for a totally different algorithm https://issues.apache.org/jira/browse/SPARK-1706 In current implementation, the user has to start multiple workers in a server for starting multiple executors in a server, which introduces additional overhead due to the more JVM processes... In this patch, I changed the scheduling logic in master to enable the user to start multiple executor processes within the same JVM process. 1. user configure spark.executor.maxCoreNumPerExecutor to suggest the maximum core he/she would like to allocate to each executor 2. Master assigns the executors to the workers with the major consideration on the memoryPerExecutor and the worker.freeMemory, and tries to allocate as many as possible cores to the executor ```min(min(memoryPerExecutor, worker.freeCore), maxLeftCoreToAssign)``` where ```maxLeftCoreToAssign = maxExecutorCanAssign * maxCoreNumPerExecutor``` --------------------------------------- Other small changes include change memoryPerSlave in ApplicationDescription to memoryPerExecutor, as "Slave" is overrided to represent both worker and executor in the documents... (we have some discussion on this before?) Author: CodingCat <zhunansjtu@gmail.com> Closes #731 from CodingCat/SPARK-1706-2 and squashes the following commits: 6dee808 [CodingCat] change filter predicate fbeb7e5 [CodingCat] address the comments 940cb42 [CodingCat] avoid unnecessary allocation b8ca561 [CodingCat] revert a change 45967b4 [CodingCat] remove unused method 2eeff77 [CodingCat] stylistic fixes 12a1b32 [CodingCat] change the semantic of coresPerExecutor to exact core number f035423 [CodingCat] stylistic fix d9c1685 [CodingCat] remove unused var f595bd6 [CodingCat] recover some unintentional changes 63b3df9 [CodingCat] change the description of the parameter in the submit script 4cf61f1 [CodingCat] improve the code and docs ff011e2 [CodingCat] start multiple executors on the worker by rewriting startExeuctor logic 2c2bcc5 [CodingCat] fix wrong usage info 497ec2c [CodingCat] address andrew's comments 878402c [CodingCat] change the launching executor code f64a28d [CodingCat] typo fix 387f4ec [CodingCat] bug fix 35c462c [CodingCat] address Andrew's comments 0b64fea [CodingCat] fix compilation issue 19d3da7 [CodingCat] address the comments 5b81466 [CodingCat] remove outdated comments ec7d421 [CodingCat] test commit e5efabb [CodingCat] more java docs and consolidate canUse function a26096d [CodingCat] stylistic fix a5d629a [CodingCat] java doc b34ec0c [CodingCat] make master support multiple executors per worker
2015-04-14 16:32:06 -04:00
</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>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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>
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:
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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>
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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>0.5.0</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
</tr>
<tr>
[SPARK-30911][CORE][DOC] Add version information to the configuration of Status ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Status`. 2.Update the docs of `Status`. 3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847 I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |   spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |   spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |   spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27848 from beliefer/add-version-to-status-config. Lead-authored-by: beliefer <beliefer@163.com> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 22:03:47 -04:00
<td><code>spark.executor.heartbeatInterval</code></td>
<td>10s</td>
<td>
Interval between each executor's heartbeats to the driver. Heartbeats let
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
the driver know that the executor is still alive and update it with metrics for in-progress
tasks. spark.executor.heartbeatInterval should be significantly less than
[SPARK-30911][CORE][DOC] Add version information to the configuration of Status ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Status`. 2.Update the docs of `Status`. 3.By the way supplementary documentation about https://github.com/apache/spark/pull/27847 I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.appStateStore.asyncTracking.enable | 2.3.0 | SPARK-20653 | 772e4648d95bda3353723337723543c741ea8476#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.period | 2.3.0 | SPARK-20644 | c7f38e5adb88d43ef60662c5d6ff4e7a95bff580#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.liveUpdate.minFlushPeriod | 2.4.2 | SPARK-27394 | a8a2ba11ac10051423e58920062b50f328b06421#diff-9ab674b7af7b2097f7d28cb6f5fd1e8c |   spark.ui.retainedJobs | 1.2.0 | SPARK-2321 | 9530316887612dca060a128fca34dd5a6ab2a9a9#diff-1f32bcb61f51133bd0959a4177a066a5 |   spark.ui.retainedStages | 0.9.0 | None | 112c0a1776bbc866a1026a9579c6f72f293414c4#diff-1f32bcb61f51133bd0959a4177a066a5 | 0.9.0-incubating-SNAPSHOT spark.ui.retainedTasks | 2.0.1 | SPARK-15083 | 55db26245d69bb02b7d7d5f25029b1a1cd571644#diff-6bdad48cfc34314e89599655442ff210 |   spark.ui.retainedDeadExecutors | 2.0.0 | SPARK-7729 | 9f4263392e492b5bc0acecec2712438ff9a257b7#diff-a0ba36f9b1f9829bf3c4689b05ab6cf2 |   spark.ui.dagGraph.retainedRootRDDs | 2.1.0 | SPARK-17171 | cc87280fcd065b01667ca7a59a1a32c7ab757355#diff-3f492c527ea26679d4307041b28455b8 |   spark.metrics.appStatusSource.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-9f796ae06b0272c1f0a012652a5b68d0 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27848 from beliefer/add-version-to-status-config. Lead-authored-by: beliefer <beliefer@163.com> Co-authored-by: Jiaan Geng <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 22:03:47 -04:00
spark.network.timeout
</td>
<td>1.1.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.files.fetchTimeout</code></td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
<td>60s</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
Communication timeout to use when fetching files added through SparkContext.addFile() from
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
the driver.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.0.0</td>
</tr>
<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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.2.2</td>
</tr>
<tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td><code>spark.files.overwrite</code></td>
<td>false</td>
<td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.0.0</td>
</tr>
<tr>
<td><code>spark.files.maxPartitionBytes</code></td>
<td>134217728 (128 MiB)</td>
<td>
The maximum number of bytes to pack into a single partition when reading files.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.1.0</td>
</tr>
<tr>
<td><code>spark.files.openCostInBytes</code></td>
<td>4194304 (4 MiB)</td>
<td>
The estimated cost to open a file, measured by the number of bytes could be scanned at the same
time. This is used when putting multiple files into a partition. It is better to overestimate,
then the partitions with small files will be faster than partitions with bigger files.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.1.0</td>
</tr>
<tr>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04: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
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
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
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
are not affected by these issues.
</td>
<td>1.0.3</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<tr>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<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)
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
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>
<td>1.0.1</td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<tr>
<td><code>spark.storage.memoryMapThreshold</code></td>
[SPARK-5932] [CORE] Use consistent naming for size properties I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5574 from ilganeli/SPARK-5932 and squashes the following commits: 11f6999 [Ilya Ganelin] Nit fixes 49a8720 [Ilya Ganelin] Whitespace fix 2ab886b [Ilya Ganelin] Scala style fc85733 [Ilya Ganelin] Got rid of floating point math 852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes 9ee779c [Ilya Ganelin] Simplified fraction matches 22413b1 [Ilya Ganelin] Made MAX private 3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values. e428049 [Ilya Ganelin] resolving merge conflict 8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles 84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer fe286b4 [Ilya Ganelin] Resolved merge conflict c7803cd [Ilya Ganelin] Empty lines 54b78b4 [Ilya Ganelin] Simplified byteUnit class 69e2f20 [Ilya Ganelin] Updates to code f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size 0f4443e [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 35a7fa7 [Ilya Ganelin] Minor formatting 928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints 5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates 7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size 2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes 1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max 0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight 475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega 851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion 747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion 09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils 5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932 db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
2015-04-28 15:18:55 -04:00
<td>2m</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td>
Size of a block above which Spark memory maps when reading a block from disk. Default unit is bytes,
unless specified otherwise. 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.
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>0.9.2</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>2.2.0</td>
</tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</table>
### Executor Metrics
<table class="table">
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
[SPARK-30812][SQL][CORE] Revise boolean config name to comply with new config naming policy ### What changes were proposed in this pull request? Revise below config names to comply with [new config naming policy](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-naming-policy-of-Spark-configs-td28875.html): SQL: * spark.sql.execution.subquery.reuse.enabled / [SPARK-27083](https://issues.apache.org/jira/browse/SPARK-27083) * spark.sql.legacy.allowNegativeScaleOfDecimal.enabled / [SPARK-30252](https://issues.apache.org/jira/browse/SPARK-30252) * spark.sql.adaptive.optimizeSkewedJoin.enabled / [SPARK-29544](https://issues.apache.org/jira/browse/SPARK-29544) * spark.sql.legacy.property.nonReserved / [SPARK-30183](https://issues.apache.org/jira/browse/SPARK-30183) * spark.sql.streaming.forceDeleteTempCheckpointLocation.enabled / [SPARK-26389](https://issues.apache.org/jira/browse/SPARK-26389) * spark.sql.analyzer.failAmbiguousSelfJoin.enabled / [SPARK-28344](https://issues.apache.org/jira/browse/SPARK-28344) * spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled / [SPARK-30074](https://issues.apache.org/jira/browse/SPARK-30074) * spark.sql.execution.pandas.arrowSafeTypeConversion / [SPARK-25811](https://issues.apache.org/jira/browse/SPARK-25811) * spark.sql.legacy.looseUpcast / [SPARK-24586](https://issues.apache.org/jira/browse/SPARK-24586) * spark.sql.legacy.arrayExistsFollowsThreeValuedLogic / [SPARK-28052](https://issues.apache.org/jira/browse/SPARK-28052) * spark.sql.sources.ignoreDataLocality.enabled / [SPARK-29189](https://issues.apache.org/jira/browse/SPARK-29189) * spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled / [SPARK-9853](https://issues.apache.org/jira/browse/SPARK-9853) CORE: * spark.eventLog.erasureCoding.enabled / [SPARK-25855](https://issues.apache.org/jira/browse/SPARK-25855) * spark.shuffle.readHostLocalDisk.enabled / [SPARK-30235](https://issues.apache.org/jira/browse/SPARK-30235) * spark.scheduler.listenerbus.logSlowEvent.enabled / [SPARK-29001](https://issues.apache.org/jira/browse/SPARK-29001) * spark.resources.coordinate.enable / [SPARK-27371](https://issues.apache.org/jira/browse/SPARK-27371) * spark.eventLog.logStageExecutorMetrics.enabled / [SPARK-23429](https://issues.apache.org/jira/browse/SPARK-23429) ### Why are the changes needed? To comply with the config naming policy. ### Does this PR introduce any user-facing change? No. Configurations listed above are all newly added in Spark 3.0. ### How was this patch tested? Pass Jenkins. Closes #27563 from Ngone51/revise_boolean_conf_name. Authored-by: yi.wu <yi.wu@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 07:39:50 -05:00
<td><code>spark.eventLog.logStageExecutorMetrics</code></td>
<td>false</td>
<td>
Whether to write per-stage peaks of executor metrics (for each executor) to the event log.
<br />
<em>Note:</em> The metrics are polled (collected) and sent in the executor heartbeat,
and this is always done; this configuration is only to determine if aggregated metric peaks
are written to the event log.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
</tr>
<td><code>spark.executor.processTreeMetrics.enabled</code></td>
<td>false</td>
<td>
Whether to collect process tree metrics (from the /proc filesystem) when collecting
executor metrics.
<br />
<em>Note:</em> The process tree metrics are collected only if the /proc filesystem
exists.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
<tr>
<td><code>spark.executor.metrics.pollingInterval</code></td>
<td>0</td>
<td>
How often to collect executor metrics (in milliseconds).
<br />
If 0, the polling is done on executor heartbeats (thus at the heartbeat interval,
specified by <code>spark.executor.heartbeatInterval</code>).
If positive, the polling is done at this interval.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>3.0.0</td>
</tr>
</table>
### Networking
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<table class="table">
[SPARK-30888][CORE][DOC] Add version information to the configuration of Network ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Network`. 2.Update the docs of `Network`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.network.crypto.saslFallback | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.network.crypto.enabled | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.remoteReadNioBufferConversion | 2.4.0 | SPARK-24307 | 2c82745686f4456c4d5c84040a431dcb5b6cb60b#diff-2b643ea78c1add0381754b1f47eec132 |   spark.network.timeout | 1.3.0 | SPARK-4688 | d3f07fd23cc26a70f44c52e24445974d4885d58a#diff-1df6b5af3d8f9f16255ff8c7a06f402f |   spark.network.timeoutInterval | 1.3.2 | SPARK-5529 | ec196ab1c7569d7ab0a50c9d7338c2835f2c84d5#diff-47779b72f095f7e7f926898fa1a425ee |   spark.rpc.askTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.connect.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.numConnectionsPerPeer | 1.6.0 | SPARK-10745 | 34a77679877bc40b58a10ec539a8da00fed7db39#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.lookupTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.message.maxSize | 2.0.0 | SPARK-7997 | bc1babd63da4ee56e6d371eb24805a5d714e8295#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.netty.dispatcher.numThreads | 1.6.0 | SPARK-11079 | 1797055dbf1d2fd7714d7c65c8d2efde2f15efc1#diff-05133dfc4bfdb6a27aa092d86ce24866 |   spark.rpc.numRetries | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.retry.wait | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27674 from beliefer/add-version-to-network-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 21:05:11 -05:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
2013-01-21 16:10:02 -05:00
<tr>
<td><code>spark.rpc.message.maxSize</code></td>
<td>128</td>
2013-01-21 16:10:02 -05:00
<td>
Maximum message size (in MiB) to allow in "control plane" communication; generally only applies to map
output size information sent between executors and the driver. Increase this if you are running
jobs with many thousands of map and reduce tasks and see messages about the RPC message size.
</td>
[SPARK-30888][CORE][DOC] Add version information to the configuration of Network ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Network`. 2.Update the docs of `Network`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.network.crypto.saslFallback | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.network.crypto.enabled | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.remoteReadNioBufferConversion | 2.4.0 | SPARK-24307 | 2c82745686f4456c4d5c84040a431dcb5b6cb60b#diff-2b643ea78c1add0381754b1f47eec132 |   spark.network.timeout | 1.3.0 | SPARK-4688 | d3f07fd23cc26a70f44c52e24445974d4885d58a#diff-1df6b5af3d8f9f16255ff8c7a06f402f |   spark.network.timeoutInterval | 1.3.2 | SPARK-5529 | ec196ab1c7569d7ab0a50c9d7338c2835f2c84d5#diff-47779b72f095f7e7f926898fa1a425ee |   spark.rpc.askTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.connect.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.numConnectionsPerPeer | 1.6.0 | SPARK-10745 | 34a77679877bc40b58a10ec539a8da00fed7db39#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.lookupTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.message.maxSize | 2.0.0 | SPARK-7997 | bc1babd63da4ee56e6d371eb24805a5d714e8295#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.netty.dispatcher.numThreads | 1.6.0 | SPARK-11079 | 1797055dbf1d2fd7714d7c65c8d2efde2f15efc1#diff-05133dfc4bfdb6a27aa092d86ce24866 |   spark.rpc.numRetries | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.retry.wait | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27674 from beliefer/add-version-to-network-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 21:05:11 -05:00
<td>2.0.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.blockManager.port</code></td>
<td>(random)</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>1.1.0</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.1.0</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
</tr>
<tr>
<td><code>spark.driver.bindAddress</code></td>
<td>(value of spark.driver.host)</td>
<td>
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
<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
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>2.1.0</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
</tr>
Merge pull request #497 from tdas/docs-update Updated Spark Streaming Programming Guide Here is the updated version of the Spark Streaming Programming Guide. This is still a work in progress, but the major changes are in place. So feedback is most welcome. In general, I have tried to make the guide to easier to understand even if the reader does not know much about Spark. The updated website is hosted here - http://www.eecs.berkeley.edu/~tdas/spark_docs/streaming-programming-guide.html The major changes are: - Overview illustrates the usecases of Spark Streaming - various input sources and various output sources - An example right after overview to quickly give an idea of what Spark Streaming program looks like - Made Java API and examples a first class citizen like Scala by using tabs to show both Scala and Java examples (similar to AMPCamp tutorial's code tabs) - Highlighted the DStream operations updateStateByKey and transform because of their powerful nature - Updated driver node failure recovery text to highlight automatic recovery in Spark standalone mode - Added information about linking and using the external input sources like Kafka and Flume - In general, reorganized the sections to better show the Basic section and the more advanced sections like Tuning and Recovery. Todos: - Links to the docs of external Kafka, Flume, etc - Illustrate window operation with figure as well as example. Author: Tathagata Das <tathagata.das1565@gmail.com> == Merge branch commits == commit 18ff10556570b39d672beeb0a32075215cfcc944 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Tue Jan 28 21:49:30 2014 -0800 Fixed a lot of broken links. commit 34a5a6008dac2e107624c7ff0db0824ee5bae45f Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Tue Jan 28 18:02:28 2014 -0800 Updated github url to use SPARK_GITHUB_URL variable. commit f338a60ae8069e0a382d2cb170227e5757cc0b7a Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Mon Jan 27 22:42:42 2014 -0800 More updates based on Patrick and Harvey's comments. commit 89a81ff25726bf6d26163e0dd938290a79582c0f Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Mon Jan 27 13:08:34 2014 -0800 Updated docs based on Patricks PR comments. commit d5b6196b532b5746e019b959a79ea0cc013a8fc3 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Sun Jan 26 20:15:58 2014 -0800 Added spark.streaming.unpersist config and info on StreamingListener interface. commit e3dcb46ab83d7071f611d9b5008ba6bc16c9f951 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Sun Jan 26 18:41:12 2014 -0800 Fixed docs on StreamingContext.getOrCreate. commit 6c29524639463f11eec721e4d17a9d7159f2944b Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Thu Jan 23 18:49:39 2014 -0800 Added example and figure for window operations, and links to Kafka and Flume API docs. commit f06b964a51bb3b21cde2ff8bdea7d9785f6ce3a9 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Wed Jan 22 22:49:12 2014 -0800 Fixed missing endhighlight tag in the MLlib guide. commit 036a7d46187ea3f2a0fb8349ef78f10d6c0b43a9 Merge: eab351d a1cd185 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Wed Jan 22 22:17:42 2014 -0800 Merge remote-tracking branch 'apache/master' into docs-update commit eab351d05c0baef1d4b549e1581310087158d78d Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Wed Jan 22 22:17:15 2014 -0800 Update Spark Streaming Programming Guide.
2014-01-29 00:51:05 -05:00
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.driver.host</code></td>
<td>(local hostname)</td>
Merge pull request #497 from tdas/docs-update Updated Spark Streaming Programming Guide Here is the updated version of the Spark Streaming Programming Guide. This is still a work in progress, but the major changes are in place. So feedback is most welcome. In general, I have tried to make the guide to easier to understand even if the reader does not know much about Spark. The updated website is hosted here - http://www.eecs.berkeley.edu/~tdas/spark_docs/streaming-programming-guide.html The major changes are: - Overview illustrates the usecases of Spark Streaming - various input sources and various output sources - An example right after overview to quickly give an idea of what Spark Streaming program looks like - Made Java API and examples a first class citizen like Scala by using tabs to show both Scala and Java examples (similar to AMPCamp tutorial's code tabs) - Highlighted the DStream operations updateStateByKey and transform because of their powerful nature - Updated driver node failure recovery text to highlight automatic recovery in Spark standalone mode - Added information about linking and using the external input sources like Kafka and Flume - In general, reorganized the sections to better show the Basic section and the more advanced sections like Tuning and Recovery. Todos: - Links to the docs of external Kafka, Flume, etc - Illustrate window operation with figure as well as example. Author: Tathagata Das <tathagata.das1565@gmail.com> == Merge branch commits == commit 18ff10556570b39d672beeb0a32075215cfcc944 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Tue Jan 28 21:49:30 2014 -0800 Fixed a lot of broken links. commit 34a5a6008dac2e107624c7ff0db0824ee5bae45f Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Tue Jan 28 18:02:28 2014 -0800 Updated github url to use SPARK_GITHUB_URL variable. commit f338a60ae8069e0a382d2cb170227e5757cc0b7a Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Mon Jan 27 22:42:42 2014 -0800 More updates based on Patrick and Harvey's comments. commit 89a81ff25726bf6d26163e0dd938290a79582c0f Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Mon Jan 27 13:08:34 2014 -0800 Updated docs based on Patricks PR comments. commit d5b6196b532b5746e019b959a79ea0cc013a8fc3 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Sun Jan 26 20:15:58 2014 -0800 Added spark.streaming.unpersist config and info on StreamingListener interface. commit e3dcb46ab83d7071f611d9b5008ba6bc16c9f951 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Sun Jan 26 18:41:12 2014 -0800 Fixed docs on StreamingContext.getOrCreate. commit 6c29524639463f11eec721e4d17a9d7159f2944b Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Thu Jan 23 18:49:39 2014 -0800 Added example and figure for window operations, and links to Kafka and Flume API docs. commit f06b964a51bb3b21cde2ff8bdea7d9785f6ce3a9 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Wed Jan 22 22:49:12 2014 -0800 Fixed missing endhighlight tag in the MLlib guide. commit 036a7d46187ea3f2a0fb8349ef78f10d6c0b43a9 Merge: eab351d a1cd185 Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Wed Jan 22 22:17:42 2014 -0800 Merge remote-tracking branch 'apache/master' into docs-update commit eab351d05c0baef1d4b549e1581310087158d78d Author: Tathagata Das <tathagata.das1565@gmail.com> Date: Wed Jan 22 22:17:15 2014 -0800 Update Spark Streaming Programming Guide.
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.
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
This is used for communicating with the executors and the standalone Master.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>0.7.0</td>
</tr>
2013-07-28 02:08:00 -04:00
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847 and https://github.com/apache/spark/pull/27852. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.metrics.namespace | 2.1.0 | SPARK-5847 | 70f846a313061e4db6174e0dc6c12c8c806ccf78#diff-6bdad48cfc34314e89599655442ff210 | spark.metrics.conf | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-7ea2624e832b166ca27cd4baca8691d9 |   spark.metrics.executorMetricsSource.enabled | 3.0.0 | SPARK-27189 | 729f43f499f3dd2718c0b28d73f2ca29cc811eac#diff-6bdad48cfc34314e89599655442ff210 |   spark.metrics.staticSources.enabled | 3.0.0 | SPARK-30060 | 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.driver.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.pyspark.python | 2.1.0 | SPARK-13081 | 7a9e25c38380e6c62080d62ad38a4830e44fe753#diff-6bdad48cfc34314e89599655442ff210 |   spark.history.ui.maxApplications | 2.0.1 | SPARK-17243 | 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.enabled | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keygen.algorithm | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.keySizeBits | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.io.encryption.commons.config.* | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6 |   spark.io.crypto.cipher.transformation | 2.1.0 | SPARK-5682 | 4b4e329e49f8af28fa6301bd06c48d7097eaf9e6#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.host | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.port | 0.7.0 | None | 02a6761589c35f15f1a6e3b63a7964ba057d3ba6#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.driver.supervise | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.bindAddress | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.blockManager.port | 1.1.0 | SPARK-2157 | 31090e43ca91f687b0bc6e25c824dc25bd7027cd#diff-2b643ea78c1add0381754b1f47eec132 |   spark.driver.blockManager.port | 2.1.0 | SPARK-4563 | 2cd1bfa4f0c6625b0ab1dbeba2b9586b9a6a9f42#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreCorruptFiles | 2.1.0 | SPARK-17850 | 47776e7c0c68590fe446cef910900b1aaead06f9#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.ignoreMissingFiles | 2.4.0 | SPARK-22676 | ed4101d29f50d54fd7846421e4c00e9ecd3599d0#diff-6bdad48cfc34314e89599655442ff210 |   spark.log.callerContext | 2.2.0 | SPARK-16759 | 3af894511be6fcc17731e28b284dba432fe911f5#diff-6bdad48cfc34314e89599655442ff210 | In branch-2.2 but pom.xml is 2.1.0-SNAPSHOT spark.files.maxPartitionBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.openCostInBytes | 2.1.0 | SPARK-16575 | c8879bf1ee2af9ccd5d5656571d931d2fc1da024#diff-6bdad48cfc34314e89599655442ff210 |   spark.hadoopRDD.ignoreEmptySplits | 2.3.0 | SPARK-22233 | 0fa10666cf75e3c4929940af49c8a6f6ea874759#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.regex | 2.1.2 | SPARK-18535 and SPARK-19720 | 444cca14d7ac8c5ab5d7e9d080b11f4d6babe3bf#diff-6bdad48cfc34314e89599655442ff210 |   spark.redaction.string.regex | 2.2.0 | SPARK-20070 | 91fa80fe8a2480d64c430bd10f97b3d44c007bcc#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.secretBitLength | 1.6.0 | SPARK-11073 | f8d93edec82eedab59d50aec06ca2de7e4cf14f6#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate | 1.0.0 | SPARK-1189 | 7edbea41b43e0dc11a2de156be220db8b7952d01#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 |   spark.authenticate.enableSaslEncryption | 1.4.0 | SPARK-6229 | 38d4e9e446b425ca6a8fe8d8080f387b08683842#diff-afd88f677ec5ff8b5e96a5cbbe00cd98 | spark.authenticate.secret.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.driver.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.authenticate.secret.executor.file | 3.0.0 | SPARK-26239 | 57d6fbfa8c803ce1791e7be36aba0219a1fcaa63#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.write.chunkSize | 2.3.0 | SPARK-21527 | 574ef6c987c636210828e96d2f797d8f10aff05e#diff-6bdad48cfc34314e89599655442ff210 |   spark.checkpoint.compress | 2.2.0 | SPARK-19525 | 1405862382185e04b09f84af18f82f2f0295a755#diff-6bdad48cfc34314e89599655442ff210 |   spark.rdd.checkpoint.cachePreferredLocsExpireTime | 3.0.0 | SPARK-29182 | 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.accurateBlockThreshold | 2.2.1 | SPARK-20801 | 81f63c8923416014d5c6bc227dd3c4e2a62bac8e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.timeout | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.registration.maxAttempts | 2.3.0 | SPARK-20640 | d107b3b910d8f434fb15b663a9db4c2dfe0a9f43#diff-6bdad48cfc34314e89599655442ff210 |   spark.reducer.maxBlocksInFlightPerAddress | 2.2.1 | SPARK-21243 | 88dccda393bc79dc6032f71b6acf8eb2b4b152be#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.maxRemoteBlockSizeFetchToMem | 3.0.0 | SPARK-26700 | d8613571bc1847775dd5c1945757279234cb388c#diff-6bdad48cfc34314e89599655442ff210 | spark.taskMetrics.trackUpdatedBlockStatuses | 2.3.0 | SPARK-20923 | 5b5a69bea9de806e2c39b04b248ee82a7b664d7b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.io.plugin.class | 3.0.0 | SPARK-28209 | abef84a868e9e15f346eea315bbab0ec8ac8e389#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.file.buffer | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-ecdafc46b901740134261d2cab24ccd9 |   spark.shuffle.unsafe.file.output.buffer | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.diskWriteBufferSize | 2.3.0 | SPARK-20950 | 565e7a8d4ae7879ee704fb94ae9b3da31e202d7e#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryCheckPeriod | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.storage.unrollMemoryGrowthFactor | 2.3.0 | SPARK-21923 | a11db942aaf4c470a85f8a1b180f034f7a584254#diff-6bdad48cfc34314e89599655442ff210 |   spark.yarn.dist.forceDownloadSchemes | 2.3.0 | SPARK-21917 | 8319432af60b8e1dc00f08d794f7d80591e24d0c#diff-6bdad48cfc34314e89599655442ff210 |   spark.extraListeners | 1.3.0 | SPARK-5411 | 47e4d579eb4a9aab8e0dd9c1400394d80c8d0388#diff-364713d7776956cb8b0a771e9b62f82d |   spark.shuffle.spill.numElementsForceSpillThreshold | 1.6.0 | SPARK-10708 | f6d06adf05afa9c5386dc2396c94e7a98730289f#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.mapOutput.parallelAggregationThreshold | 2.3.0 | SPARK-22537 | efd0036ec88bdc385f5a9ea568d2e2bbfcda2912#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.maxResultSize | 1.2.0 | SPARK-3466 | 6181577e9935f46b646ba3925b873d031aa3d6ba#diff-d239aee594001f8391676e1047a0381e | spark.security.credentials.renewalRatio | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.security.credentials.retryWait | 2.4.0 | SPARK-23361 | 5fa438471110afbf4e2174df449ac79e292501f8#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sort.initialBufferSize | 2.1.0 | SPARK-15958 | bf665a958631125a1670504ef5966ef1a0e14798#diff-a1d00506391c1c4b2209f9bbff590c5b | On branch-2.1, but in pom.xml it is 2.0.0-SNAPSHOT spark.shuffle.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.shuffle.spill.compress | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-2b643ea78c1add0381754b1f47eec132 |   spark.shuffle.mapStatus.compression.codec | 3.0.0 | SPARK-29939 | 456cfe6e4693efd26d64f089d53c4e01bf8150a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.spill.initialMemoryThreshold | 1.1.1 | SPARK-4480 | 16bf5f3d17624db2a96c921fe8a1e153cdafb06c#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.spill.batchSize | 0.9.0 | None | c3816de5040e3c48e58ed4762d2f4eb606812938#diff-a470b9812a5ac8c37d732da7d9fbe39a | spark.shuffle.sort.bypassMergeThreshold | 1.1.1 | SPARK-2787 | 0f2274f8ed6131ad17326e3fff7f7e093863b72d#diff-31417c461d8901d8e08167b0cbc344c1 |   spark.shuffle.manager | 1.1.0 | SPARK-2044 | 508fd371d6dbb826fd8a00787d347235b549e189#diff-60df49b5d3c59f2c4540fa16a90033a1 |   spark.shuffle.reduceLocality.enabled | 1.5.0 | SPARK-2774 | 96a7c888d806adfdb2c722025a1079ed7eaa2052#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.shuffle.mapOutput.minSizeForBroadcast | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.mapOutput.dispatcher.numThreads | 2.0.0 | SPARK-1239 | d98dd72e7baeb59eacec4fefd66397513a607b2f#diff-609c3f8c26150ca96a94cd27146a809b |   spark.shuffle.detectCorrupt | 2.2.0 | SPARK-4105 | cf33a86285629abe72c1acf235b8bfa6057220a8#diff-eb30a71e0d04150b8e0b64929852e38b | spark.shuffle.detectCorrupt.useExtraMemory | 3.0.0 | SPARK-26089 | 688b0c01fac0db80f6473181673a89f1ce1be65b#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.sync | 0.8.0 | None | 31da065b1d08c1fad5283e4bcf8e0ed01818c03e#diff-ad46ed23fcc3fa87f30d05204917b917 |   spark.shuffle.unsafe.fastMergeEnabled | 1.4.0 | SPARK-7081 | c53ebea9db418099df50f9adc1a18cee7849cd97#diff-642ce9f439435408382c3ac3b5c5e0a0 |   spark.shuffle.sort.useRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-3eedc75de4787b842477138d8cc7f150 |   spark.shuffle.minNumPartitionsToHighlyCompress | 2.4.0 | SPARK-24519 | 39dfaf2fd167cafc84ec9cc637c114ed54a331e3#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.useOldFetchProtocol | 3.0.0 | SPARK-25341 | f725d472f51fb80c6ce1882ec283ff69bafb0de4#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.readHostLocalDisk | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27913 from beliefer/add-version-to-core-config-part-three. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-15 21:08:07 -04:00
<td>0.7.0</td>
2013-07-28 02:08:00 -04:00
</tr>
<tr>
<td><code>spark.rpc.io.backLog</code></td>
<td>64</td>
<td>
Length of the accept queue for the RPC server. For large applications, this value may
need to be increased, so that incoming connections are not dropped when a large number of
connections arrives in a short period of time.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.network.timeout</code></td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
<td>120s</td>
<td>
Default timeout for all network interactions. This config will be used in place of
<code>spark.core.connection.ack.wait.timeout</code>,
<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.
</td>
[SPARK-30888][CORE][DOC] Add version information to the configuration of Network ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Network`. 2.Update the docs of `Network`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.network.crypto.saslFallback | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.network.crypto.enabled | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.remoteReadNioBufferConversion | 2.4.0 | SPARK-24307 | 2c82745686f4456c4d5c84040a431dcb5b6cb60b#diff-2b643ea78c1add0381754b1f47eec132 |   spark.network.timeout | 1.3.0 | SPARK-4688 | d3f07fd23cc26a70f44c52e24445974d4885d58a#diff-1df6b5af3d8f9f16255ff8c7a06f402f |   spark.network.timeoutInterval | 1.3.2 | SPARK-5529 | ec196ab1c7569d7ab0a50c9d7338c2835f2c84d5#diff-47779b72f095f7e7f926898fa1a425ee |   spark.rpc.askTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.connect.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.numConnectionsPerPeer | 1.6.0 | SPARK-10745 | 34a77679877bc40b58a10ec539a8da00fed7db39#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.lookupTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.message.maxSize | 2.0.0 | SPARK-7997 | bc1babd63da4ee56e6d371eb24805a5d714e8295#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.netty.dispatcher.numThreads | 1.6.0 | SPARK-11079 | 1797055dbf1d2fd7714d7c65c8d2efde2f15efc1#diff-05133dfc4bfdb6a27aa092d86ce24866 |   spark.rpc.numRetries | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.retry.wait | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27674 from beliefer/add-version-to-network-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 21:05:11 -05:00
<td>1.3.0</td>
</tr>
<tr>
<td><code>spark.network.io.preferDirectBufs</code></td>
<td>true</td>
<td>
If enabled then off-heap buffer allocations are preferred by the shared allocators.
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 to be on-heap.
[SPARK-30888][CORE][DOC] Add version information to the configuration of Network ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Network`. 2.Update the docs of `Network`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.network.crypto.saslFallback | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.network.crypto.enabled | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.remoteReadNioBufferConversion | 2.4.0 | SPARK-24307 | 2c82745686f4456c4d5c84040a431dcb5b6cb60b#diff-2b643ea78c1add0381754b1f47eec132 |   spark.network.timeout | 1.3.0 | SPARK-4688 | d3f07fd23cc26a70f44c52e24445974d4885d58a#diff-1df6b5af3d8f9f16255ff8c7a06f402f |   spark.network.timeoutInterval | 1.3.2 | SPARK-5529 | ec196ab1c7569d7ab0a50c9d7338c2835f2c84d5#diff-47779b72f095f7e7f926898fa1a425ee |   spark.rpc.askTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.connect.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.numConnectionsPerPeer | 1.6.0 | SPARK-10745 | 34a77679877bc40b58a10ec539a8da00fed7db39#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.lookupTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.message.maxSize | 2.0.0 | SPARK-7997 | bc1babd63da4ee56e6d371eb24805a5d714e8295#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.netty.dispatcher.numThreads | 1.6.0 | SPARK-11079 | 1797055dbf1d2fd7714d7c65c8d2efde2f15efc1#diff-05133dfc4bfdb6a27aa092d86ce24866 |   spark.rpc.numRetries | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.retry.wait | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27674 from beliefer/add-version-to-network-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 21:05:11 -05:00
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.port.maxRetries</code></td>
<td>16</td>
<td>
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
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
to port + maxRetries.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.1.1</td>
</tr>
<tr>
<td><code>spark.rpc.numRetries</code></td>
<td>3</td>
<td>
Number of times to retry before an RPC task gives up.
An RPC task will run at most times of this number.
</td>
[SPARK-30888][CORE][DOC] Add version information to the configuration of Network ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Network`. 2.Update the docs of `Network`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.network.crypto.saslFallback | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.network.crypto.enabled | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.remoteReadNioBufferConversion | 2.4.0 | SPARK-24307 | 2c82745686f4456c4d5c84040a431dcb5b6cb60b#diff-2b643ea78c1add0381754b1f47eec132 |   spark.network.timeout | 1.3.0 | SPARK-4688 | d3f07fd23cc26a70f44c52e24445974d4885d58a#diff-1df6b5af3d8f9f16255ff8c7a06f402f |   spark.network.timeoutInterval | 1.3.2 | SPARK-5529 | ec196ab1c7569d7ab0a50c9d7338c2835f2c84d5#diff-47779b72f095f7e7f926898fa1a425ee |   spark.rpc.askTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.connect.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.numConnectionsPerPeer | 1.6.0 | SPARK-10745 | 34a77679877bc40b58a10ec539a8da00fed7db39#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.lookupTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.message.maxSize | 2.0.0 | SPARK-7997 | bc1babd63da4ee56e6d371eb24805a5d714e8295#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.netty.dispatcher.numThreads | 1.6.0 | SPARK-11079 | 1797055dbf1d2fd7714d7c65c8d2efde2f15efc1#diff-05133dfc4bfdb6a27aa092d86ce24866 |   spark.rpc.numRetries | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.retry.wait | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27674 from beliefer/add-version-to-network-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 21:05:11 -05:00
<td>1.4.0</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>
[SPARK-30888][CORE][DOC] Add version information to the configuration of Network ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Network`. 2.Update the docs of `Network`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.network.crypto.saslFallback | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.network.crypto.enabled | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.remoteReadNioBufferConversion | 2.4.0 | SPARK-24307 | 2c82745686f4456c4d5c84040a431dcb5b6cb60b#diff-2b643ea78c1add0381754b1f47eec132 |   spark.network.timeout | 1.3.0 | SPARK-4688 | d3f07fd23cc26a70f44c52e24445974d4885d58a#diff-1df6b5af3d8f9f16255ff8c7a06f402f |   spark.network.timeoutInterval | 1.3.2 | SPARK-5529 | ec196ab1c7569d7ab0a50c9d7338c2835f2c84d5#diff-47779b72f095f7e7f926898fa1a425ee |   spark.rpc.askTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.connect.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.numConnectionsPerPeer | 1.6.0 | SPARK-10745 | 34a77679877bc40b58a10ec539a8da00fed7db39#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.lookupTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.message.maxSize | 2.0.0 | SPARK-7997 | bc1babd63da4ee56e6d371eb24805a5d714e8295#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.netty.dispatcher.numThreads | 1.6.0 | SPARK-11079 | 1797055dbf1d2fd7714d7c65c8d2efde2f15efc1#diff-05133dfc4bfdb6a27aa092d86ce24866 |   spark.rpc.numRetries | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.retry.wait | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27674 from beliefer/add-version-to-network-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 21:05:11 -05:00
<td>1.4.0</td>
</tr>
<tr>
<td><code>spark.rpc.askTimeout</code></td>
<td><code>spark.network.timeout</code></td>
<td>
Duration for an RPC ask operation to wait before timing out.
</td>
[SPARK-30888][CORE][DOC] Add version information to the configuration of Network ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Network`. 2.Update the docs of `Network`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.network.crypto.saslFallback | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.network.crypto.enabled | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.remoteReadNioBufferConversion | 2.4.0 | SPARK-24307 | 2c82745686f4456c4d5c84040a431dcb5b6cb60b#diff-2b643ea78c1add0381754b1f47eec132 |   spark.network.timeout | 1.3.0 | SPARK-4688 | d3f07fd23cc26a70f44c52e24445974d4885d58a#diff-1df6b5af3d8f9f16255ff8c7a06f402f |   spark.network.timeoutInterval | 1.3.2 | SPARK-5529 | ec196ab1c7569d7ab0a50c9d7338c2835f2c84d5#diff-47779b72f095f7e7f926898fa1a425ee |   spark.rpc.askTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.connect.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.numConnectionsPerPeer | 1.6.0 | SPARK-10745 | 34a77679877bc40b58a10ec539a8da00fed7db39#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.lookupTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.message.maxSize | 2.0.0 | SPARK-7997 | bc1babd63da4ee56e6d371eb24805a5d714e8295#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.netty.dispatcher.numThreads | 1.6.0 | SPARK-11079 | 1797055dbf1d2fd7714d7c65c8d2efde2f15efc1#diff-05133dfc4bfdb6a27aa092d86ce24866 |   spark.rpc.numRetries | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.retry.wait | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27674 from beliefer/add-version-to-network-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 21:05:11 -05:00
<td>1.4.0</td>
</tr>
<tr>
<td><code>spark.rpc.lookupTimeout</code></td>
<td>120s</td>
<td>
Duration for an RPC remote endpoint lookup operation to wait before timing out.
</td>
[SPARK-30888][CORE][DOC] Add version information to the configuration of Network ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Network`. 2.Update the docs of `Network`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.network.crypto.saslFallback | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.network.crypto.enabled | 2.2.0 | SPARK-19139 | 8f3f73abc1fe62496722476460c174af0250e3fe#diff-6bdad48cfc34314e89599655442ff210 |   spark.network.remoteReadNioBufferConversion | 2.4.0 | SPARK-24307 | 2c82745686f4456c4d5c84040a431dcb5b6cb60b#diff-2b643ea78c1add0381754b1f47eec132 |   spark.network.timeout | 1.3.0 | SPARK-4688 | d3f07fd23cc26a70f44c52e24445974d4885d58a#diff-1df6b5af3d8f9f16255ff8c7a06f402f |   spark.network.timeoutInterval | 1.3.2 | SPARK-5529 | ec196ab1c7569d7ab0a50c9d7338c2835f2c84d5#diff-47779b72f095f7e7f926898fa1a425ee |   spark.rpc.askTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.connect.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.numConnectionsPerPeer | 1.6.0 | SPARK-10745 | 34a77679877bc40b58a10ec539a8da00fed7db39#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.io.threads | 1.6.0 | SPARK-6028 | 084e4e126211d74a79e8dbd2d0e604dd3c650822#diff-0c89b4a60c30a7cd2224bb64d93da942 |   spark.rpc.lookupTimeout | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.message.maxSize | 2.0.0 | SPARK-7997 | bc1babd63da4ee56e6d371eb24805a5d714e8295#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.netty.dispatcher.numThreads | 1.6.0 | SPARK-11079 | 1797055dbf1d2fd7714d7c65c8d2efde2f15efc1#diff-05133dfc4bfdb6a27aa092d86ce24866 |   spark.rpc.numRetries | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.rpc.retry.wait | 1.4.0 | SPARK-6490 | 8136810dfad12008ac300116df7bc8448740f1ae#diff-529fc5c06b9731c1fbda6f3db60b16aa |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27674 from beliefer/add-version-to-network-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-26 21:05:11 -05:00
<td>1.4.0</td>
</tr>
<tr>
<td><code>spark.core.connection.ack.wait.timeout</code></td>
<td><code>spark.network.timeout</code></td>
<td>
How long for the connection to wait for ack to occur before timing
out and giving up. To avoid unwilling timeout caused by long pause like GC,
you can set larger value.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.1.1</td>
</tr>
<tr>
<td><code>spark.network.maxRemoteBlockSizeFetchToMem</code></td>
<td>200m</td>
<td>
Remote block will be fetched to disk when size of the block is above this threshold
in bytes. This is to avoid a giant request takes too much memory. Note this
configuration will affect both shuffle fetch and block manager remote block fetch.
For users who enabled external shuffle service, this feature can only work when
external shuffle service is at least 2.3.0.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>3.0.0</td>
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</table>
### Scheduling
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<table class="table">
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.cores.max</code></td>
<td>(not set)</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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.
</td>
[SPARK-31002][CORE][DOC] Add version information to the configuration of Core ### What changes were proposed in this pull request? Add version information to the configuration of `Core`. Note: Because `Core` has a lot of configuration items, I split the items into four PR. Other PR will follows this PR. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.resources.discoveryPlugin | 3.0.0 | SPARK-30689 | 742e35f1d48c2523dda2ce21d73b7ab5ade20582#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.resourcesFile | 3.0.0 | SPARK-27835 | 6748b486a9afe8370786efb64a8c9f3470c62dcf#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.DRIVER_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraClassPath SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraJavaOptions SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.extraLibraryPath spark.driver.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.driver.cores | 1.3.0 | SPARK-1507 | 2be82b1e66cd188456bbf1e5abb13af04d1629d5#diff-4d2ab44195558d5a9d5f15b8803ef39d |   SparkLauncher.DRIVER_MEMORY | 1.1.1 | SPARK-3243 | c1ffa3e4cdfbd1f84b5c8d8de5d0fb958a19e211#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.driver.memory spark.driver.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.dfsDir | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.layout | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.persistToDfs.enabled | 3.0.0 | SPARK-25118 | 5f11e8c4cb9a5db037ac239b8fcc97f3a746e772#diff-6bdad48cfc34314e89599655442ff210 |   spark.driver.log.allowErasureCoding | 3.0.0 | SPARK-29105 | 276aaaae8d404975f8701089e9f4dfecd16e0d9f#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.enabled | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.dir | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.compress | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logBlockUpdates.enabled | 2.3.0 | SPARK-22050 | 1437e344ec0c29a44a19f4513986f5f184c44695#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.erasureCoding.enabled | 3.0.0 | SPARK-25855 | 35506dced739ef16136e9f3d5d48c638899d3cec#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.testing | 1.0.1 | None | d4c8af87994acf3707027e6fab25363f51fd4615#diff-e4a5a68c15eed95d038acfed84b0b66a |   spark.eventLog.buffer.kb | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.logStageExecutorMetrics | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.youngGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.gcMetrics.oldGenerationGarbageCollectors | 3.0.0 | SPARK-25865 | e5c502c596563dce8eb58f86e42c1aea2c51ed17#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.overwrite | 1.0.0 | SPARK-1132 | 79d07d66040f206708e14de393ab0b80020ed96a#diff-364713d7776956cb8b0a771e9b62f82d |   spark.eventLog.longForm.enabled | 2.4.0 | SPARK-23820 | 71f70130f1b2b4ec70595627f0a02a88e2c0e27d#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.enabled | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.rolling.maxFileSize | 3.0.0 | SPARK-28869 | 100fc58da54e026cda87832a10e2d06eaeccdf87#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.id | 1.2.0 | SPARK-3377 | 79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-364713d7776956cb8b0a771e9b62f82d |   SparkLauncher.EXECUTOR_EXTRA_CLASSPATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraClassPath spark.executor.heartbeat.dropZeroAccumulatorUpdates | 3.0.0 | SPARK-25449 | 9362c5cc273fdd09f9b3b512e2f6b64bcefc25ab#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.heartbeatInterval | 1.1.0 | SPARK-2099 | 8d338f64c4eda45d22ae33f61ef7928011cc2846#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.heartbeat.maxFailures | 1.6.2 | SPARK-13522 | 86bf93e65481b8fe5d7532ca6d4cd29cafc9e9dd#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.executor.processTreeMetrics.enabled | 3.0.0 | SPARK-27324 | 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.metrics.pollingInterval | 3.0.0 | SPARK-26329 | 80ab19b9fd268adfc419457f12b99a5da7b6d1c7#diff-6bdad48cfc34314e89599655442ff210 |   SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraJavaOptions SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.extraLibraryPath spark.executor.userClassPathFirst | 1.3.0 | SPARK-2996 | 6a1e0f967286945db13d94aeb6ed19f0a347c236#diff-529fc5c06b9731c1fbda6f3db60b16aa |   SparkLauncher.EXECUTOR_CORES | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.executor.cores SparkLauncher.EXECUTOR_MEMORY | 0.7.0 | None | 696eec32c982ca516c506de33f383a173bcbd131#diff-4f50ad37deb6742ad45472636c9a870b | spark.executor.memory spark.executor.memoryOverhead | 2.3.0 | SPARK-22646 | 3f4060c340d6bac412e8819c4388ccba226efcf3#diff-6bdad48cfc34314e89599655442ff210 |   spark.cores.max | 0.6.0 | None | 0a472840030e4e7e84fe748f7bfa49f1ece599c5#diff-b6cc54c092b861f645c3cd69ea0f91e2 |   spark.memory.offHeap.enabled | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.offHeap.size | 1.6.0 | SPARK-12251 | 9870e5c7af87190167ca3845ede918671b9420ca#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.storageFraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.memory.fraction | 1.6.0 | SPARK-10983 | b3ffac5178795f2d8e7908b3e77e8e89f50b5f6f#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.storage.safetyFraction | 1.1.0 | [SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.unrollMemoryThreshold | 1.1.0 | SPARK-1777 | ecf30ee7e78ea59c462c54db0fde5328f997466c#diff-692a329b5a7fb4134c55d559457b94e4 | spark.storage.replication.proactive | 2.2.0 | SPARK-15355 | fa7c582e9442b985a0493fb1dd15b3fb9b6031b4#diff-186864190089a718680accb51de5f0d4 |   spark.storage.memoryMapThreshold | 0.9.2 | SPARK-1145 | 76339495153dd895667ad609815c887b2c8960ea#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a | spark.storage.replication.policy | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyMapper | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-186864190089a718680accb51de5f0d4 | spark.storage.cachedPeersTtl | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.maxReplicationFailures | 1.1.1 | SPARK-3495 and SPARK-3496 | be0cc9952d6c8b4cfe9ff10a761e0677cba64489#diff-2b643ea78c1add0381754b1f47eec132 |   spark.storage.replication.topologyFile | 2.1.0 | SPARK-15353 | a26afd52198523dbd51dc94053424494638c7de5#diff-e550ce522c12a31d805a7d0f41e802af |   spark.storage.exceptionOnPinLeak | 1.6.2 | SPARK-13566 | ab006523b840b1d2dbf3f5ff0a238558e7665a1e#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.storage.blockManagerTimeoutIntervalMs | 0.7.3 | None | 9085ebf3750c7d9bb7c6b5f6b4bdc5b807af93c2#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.storage.blockManagerSlaveTimeoutMs | 0.7.0 | None | 97434f49b8c029e9b78c91ec5f58557cd1b5c943#diff-2ce6374aac24d70c69182b067216e684 | spark.storage.cleanupFilesAfterExecutorExit | 2.4.0 | SPARK-24340 | 8ef167a5f9ba8a79bb7ca98a9844fe9cfcfea060#diff-916ca56b663f178f302c265b7ef38499 |   spark.diskStore.subDirectories | 0.6.0 | None | 815d6bd69a0c1ba0e94fc0785f5c3619b37f19c5#diff-e8b73c5b81c403a5e5d581f97624c510 |   spark.block.failures.beforeLocationRefresh | 2.0.0 | SPARK-13328 | ff776b2fc1cd4c571fd542dbf807e6fa3373cb34#diff-2b643ea78c1add0381754b1f47eec132 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27847 from beliefer/add-version-to-core-config-part-one. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-07 22:31:57 -05:00
<td>0.6.0</td>
</tr>
<tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td><code>spark.locality.wait</code></td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
<td>3s</td>
<td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
How long to wait to launch a data-local task before giving up and launching it
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>0.5.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.locality.wait.node</code></td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td>spark.locality.wait</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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).
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.8.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.locality.wait.process</code></td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td>spark.locality.wait</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.8.0</td>
</tr>
<tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td><code>spark.locality.wait.rack</code></td>
<td>spark.locality.wait</td>
<td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
Customize the locality wait for rack locality.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.8.0</td>
</tr>
Spark-939: allow user jars to take precedence over spark jars I still need to do a small bit of re-factoring [mostly the one Java file I'll switch it back to a Scala file and use it in both the close loaders], but comments on other things I should do would be great. Author: Holden Karau <holden@pigscanfly.ca> Closes #217 from holdenk/spark-939-allow-user-jars-to-take-precedence-over-spark-jars and squashes the following commits: cf0cac9 [Holden Karau] Fix the executorclassloader 1955232 [Holden Karau] Fix long line in TestUtils 8f89965 [Holden Karau] Fix tests for new class name 7546549 [Holden Karau] CR feedback, merge some of the testutils methods down, rename the classloader 644719f [Holden Karau] User the class generator for the repl class loader tests too f0b7114 [Holden Karau] Fix the core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala tests 204b199 [Holden Karau] Fix the generated classes 9f68f10 [Holden Karau] Start rewriting the ExecutorURLClassLoaderSuite to not use the hard coded classes 858aba2 [Holden Karau] Remove a bunch of test junk 261aaee [Holden Karau] simplify executorurlclassloader a bit 7a7bf5f [Holden Karau] CR feedback d4ae848 [Holden Karau] rewrite component into scala aa95083 [Holden Karau] CR feedback 7752594 [Holden Karau] re-add https comment a0ef85a [Holden Karau] Fix style issues 125ea7f [Holden Karau] Easier to just remove those files, we don't need them bb8d179 [Holden Karau] Fix issues with the repl class loader 241b03d [Holden Karau] fix my rat excludes a343350 [Holden Karau] Update rat-excludes and remove a useless file d90d217 [Holden Karau] Fix fall back with custom class loader and add a test for it 4919bf9 [Holden Karau] Fix parent calling class loader issue 8a67302 [Holden Karau] Test are good 9e2d236 [Holden Karau] It works comrade 691ee00 [Holden Karau] It works ish dc4fe44 [Holden Karau] Does not depend on being in my home directory 47046ff [Holden Karau] Remove bad import' 22d83cb [Holden Karau] Add a test suite for the executor url class loader suite 7ef4628 [Holden Karau] Clean up 792d961 [Holden Karau] Almost works 16aecd1 [Holden Karau] Doesn't quite work 8d2241e [Holden Karau] Adda FakeClass for testing ClassLoader precedence options 648b559 [Holden Karau] Both class loaders compile. Now for testing e1d9f71 [Holden Karau] One loader workers.
2014-04-09 01:29:21 -04:00
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.scheduler.maxRegisteredResourcesWaitingTime</code></td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
<td>30s</td>
Spark-939: allow user jars to take precedence over spark jars I still need to do a small bit of re-factoring [mostly the one Java file I'll switch it back to a Scala file and use it in both the close loaders], but comments on other things I should do would be great. Author: Holden Karau <holden@pigscanfly.ca> Closes #217 from holdenk/spark-939-allow-user-jars-to-take-precedence-over-spark-jars and squashes the following commits: cf0cac9 [Holden Karau] Fix the executorclassloader 1955232 [Holden Karau] Fix long line in TestUtils 8f89965 [Holden Karau] Fix tests for new class name 7546549 [Holden Karau] CR feedback, merge some of the testutils methods down, rename the classloader 644719f [Holden Karau] User the class generator for the repl class loader tests too f0b7114 [Holden Karau] Fix the core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala tests 204b199 [Holden Karau] Fix the generated classes 9f68f10 [Holden Karau] Start rewriting the ExecutorURLClassLoaderSuite to not use the hard coded classes 858aba2 [Holden Karau] Remove a bunch of test junk 261aaee [Holden Karau] simplify executorurlclassloader a bit 7a7bf5f [Holden Karau] CR feedback d4ae848 [Holden Karau] rewrite component into scala aa95083 [Holden Karau] CR feedback 7752594 [Holden Karau] re-add https comment a0ef85a [Holden Karau] Fix style issues 125ea7f [Holden Karau] Easier to just remove those files, we don't need them bb8d179 [Holden Karau] Fix issues with the repl class loader 241b03d [Holden Karau] fix my rat excludes a343350 [Holden Karau] Update rat-excludes and remove a useless file d90d217 [Holden Karau] Fix fall back with custom class loader and add a test for it 4919bf9 [Holden Karau] Fix parent calling class loader issue 8a67302 [Holden Karau] Test are good 9e2d236 [Holden Karau] It works comrade 691ee00 [Holden Karau] It works ish dc4fe44 [Holden Karau] Does not depend on being in my home directory 47046ff [Holden Karau] Remove bad import' 22d83cb [Holden Karau] Add a test suite for the executor url class loader suite 7ef4628 [Holden Karau] Clean up 792d961 [Holden Karau] Almost works 16aecd1 [Holden Karau] Doesn't quite work 8d2241e [Holden Karau] Adda FakeClass for testing ClassLoader precedence options 648b559 [Holden Karau] Both class loaders compile. Now for testing e1d9f71 [Holden Karau] One loader workers.
2014-04-09 01:29:21 -04:00
<td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
Maximum amount of time to wait for resources to register before scheduling begins.
Spark-939: allow user jars to take precedence over spark jars I still need to do a small bit of re-factoring [mostly the one Java file I'll switch it back to a Scala file and use it in both the close loaders], but comments on other things I should do would be great. Author: Holden Karau <holden@pigscanfly.ca> Closes #217 from holdenk/spark-939-allow-user-jars-to-take-precedence-over-spark-jars and squashes the following commits: cf0cac9 [Holden Karau] Fix the executorclassloader 1955232 [Holden Karau] Fix long line in TestUtils 8f89965 [Holden Karau] Fix tests for new class name 7546549 [Holden Karau] CR feedback, merge some of the testutils methods down, rename the classloader 644719f [Holden Karau] User the class generator for the repl class loader tests too f0b7114 [Holden Karau] Fix the core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala tests 204b199 [Holden Karau] Fix the generated classes 9f68f10 [Holden Karau] Start rewriting the ExecutorURLClassLoaderSuite to not use the hard coded classes 858aba2 [Holden Karau] Remove a bunch of test junk 261aaee [Holden Karau] simplify executorurlclassloader a bit 7a7bf5f [Holden Karau] CR feedback d4ae848 [Holden Karau] rewrite component into scala aa95083 [Holden Karau] CR feedback 7752594 [Holden Karau] re-add https comment a0ef85a [Holden Karau] Fix style issues 125ea7f [Holden Karau] Easier to just remove those files, we don't need them bb8d179 [Holden Karau] Fix issues with the repl class loader 241b03d [Holden Karau] fix my rat excludes a343350 [Holden Karau] Update rat-excludes and remove a useless file d90d217 [Holden Karau] Fix fall back with custom class loader and add a test for it 4919bf9 [Holden Karau] Fix parent calling class loader issue 8a67302 [Holden Karau] Test are good 9e2d236 [Holden Karau] It works comrade 691ee00 [Holden Karau] It works ish dc4fe44 [Holden Karau] Does not depend on being in my home directory 47046ff [Holden Karau] Remove bad import' 22d83cb [Holden Karau] Add a test suite for the executor url class loader suite 7ef4628 [Holden Karau] Clean up 792d961 [Holden Karau] Almost works 16aecd1 [Holden Karau] Doesn't quite work 8d2241e [Holden Karau] Adda FakeClass for testing ClassLoader precedence options 648b559 [Holden Karau] Both class loaders compile. Now for testing e1d9f71 [Holden Karau] One loader workers.
2014-04-09 01:29:21 -04:00
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.1.1</td>
Spark-939: allow user jars to take precedence over spark jars I still need to do a small bit of re-factoring [mostly the one Java file I'll switch it back to a Scala file and use it in both the close loaders], but comments on other things I should do would be great. Author: Holden Karau <holden@pigscanfly.ca> Closes #217 from holdenk/spark-939-allow-user-jars-to-take-precedence-over-spark-jars and squashes the following commits: cf0cac9 [Holden Karau] Fix the executorclassloader 1955232 [Holden Karau] Fix long line in TestUtils 8f89965 [Holden Karau] Fix tests for new class name 7546549 [Holden Karau] CR feedback, merge some of the testutils methods down, rename the classloader 644719f [Holden Karau] User the class generator for the repl class loader tests too f0b7114 [Holden Karau] Fix the core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala tests 204b199 [Holden Karau] Fix the generated classes 9f68f10 [Holden Karau] Start rewriting the ExecutorURLClassLoaderSuite to not use the hard coded classes 858aba2 [Holden Karau] Remove a bunch of test junk 261aaee [Holden Karau] simplify executorurlclassloader a bit 7a7bf5f [Holden Karau] CR feedback d4ae848 [Holden Karau] rewrite component into scala aa95083 [Holden Karau] CR feedback 7752594 [Holden Karau] re-add https comment a0ef85a [Holden Karau] Fix style issues 125ea7f [Holden Karau] Easier to just remove those files, we don't need them bb8d179 [Holden Karau] Fix issues with the repl class loader 241b03d [Holden Karau] fix my rat excludes a343350 [Holden Karau] Update rat-excludes and remove a useless file d90d217 [Holden Karau] Fix fall back with custom class loader and add a test for it 4919bf9 [Holden Karau] Fix parent calling class loader issue 8a67302 [Holden Karau] Test are good 9e2d236 [Holden Karau] It works comrade 691ee00 [Holden Karau] It works ish dc4fe44 [Holden Karau] Does not depend on being in my home directory 47046ff [Holden Karau] Remove bad import' 22d83cb [Holden Karau] Add a test suite for the executor url class loader suite 7ef4628 [Holden Karau] Clean up 792d961 [Holden Karau] Almost works 16aecd1 [Holden Karau] Doesn't quite work 8d2241e [Holden Karau] Adda FakeClass for testing ClassLoader precedence options 648b559 [Holden Karau] Both class loaders compile. Now for testing e1d9f71 [Holden Karau] One loader workers.
2014-04-09 01:29:21 -04:00
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<tr>
<td><code>spark.scheduler.minRegisteredResourcesRatio</code></td>
[SPARK-18278][SCHEDULER] Spark on Kubernetes - Basic Scheduler Backend ## What changes were proposed in this pull request? This is a stripped down version of the `KubernetesClusterSchedulerBackend` for Spark with the following components: - Static Allocation of Executors - Executor Pod Factory - Executor Recovery Semantics It's step 1 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935). This addition is covered by the [SPIP vote](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) which passed on Aug 31 . ## How was this patch tested? - The patch contains unit tests which are passing. - Manual testing: `./build/mvn -Pkubernetes clean package` succeeded. - It is a **subset** of the entire changelist hosted in http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations. - There is integration testing enabled in the fork currently [hosted by PepperData](spark-k8s-jenkins.pepperdata.org:8080) which is being moved over to RiseLAB CI. - Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html cc rxin felixcheung mateiz (shepherd) k8s-big-data SIG members & contributors: mccheah ash211 ssuchter varunkatta kimoonkim erikerlandson liyinan926 tnachen ifilonenko Author: Yinan Li <liyinan926@gmail.com> Author: foxish <ramanathana@google.com> Author: mcheah <mcheah@palantir.com> Closes #19468 from foxish/spark-kubernetes-3.
2017-11-29 02:02:09 -05:00
<td>0.8 for KUBERNETES mode; 0.8 for YARN mode; 0.0 for standalone mode and Mesos coarse-grained mode</td>
[SPARK-1946] Submit tasks after (configured ratio) executors have been registered Because submitting tasks and registering executors are asynchronous, in most situation, early stages' tasks run without preferred locality. A simple solution is sleeping few seconds in application, so that executors have enough time to register. The PR add 2 configuration properties to make TaskScheduler submit tasks after a few of executors have been registered. \# Submit tasks only after (registered executors / total executors) arrived the ratio, default value is 0 spark.scheduler.minRegisteredExecutorsRatio = 0.8 \# Whatever minRegisteredExecutorsRatio is arrived, submit tasks after the maxRegisteredWaitingTime(millisecond), default value is 30000 spark.scheduler.maxRegisteredExecutorsWaitingTime = 5000 Author: li-zhihui <zhihui.li@intel.com> Closes #900 from li-zhihui/master and squashes the following commits: b9f8326 [li-zhihui] Add logs & edit docs 1ac08b1 [li-zhihui] Add new configs to user docs 22ead12 [li-zhihui] Move waitBackendReady to postStartHook c6f0522 [li-zhihui] Bug fix: numExecutors wasn't set & use constant DEFAULT_NUMBER_EXECUTORS 4d6d847 [li-zhihui] Move waitBackendReady to TaskSchedulerImpl.start & some code refactor 0ecee9a [li-zhihui] Move waitBackendReady from DAGScheduler.submitStage to TaskSchedulerImpl.submitTasks 4261454 [li-zhihui] Add docs for new configs & code style ce0868a [li-zhihui] Code style, rename configuration property name of minRegisteredRatio & maxRegisteredWaitingTime 6cfb9ec [li-zhihui] Code style, revert default minRegisteredRatio of yarn to 0, driver get --num-executors in yarn/alpha 812c33c [li-zhihui] Fix driver lost --num-executors option in yarn-cluster mode e7b6272 [li-zhihui] support yarn-cluster 37f7dc2 [li-zhihui] support yarn mode(percentage style) 3f8c941 [li-zhihui] submit stage after (configured ratio of) executors have been registered
2014-07-14 16:32:49 -04:00
<td>
The minimum ratio of registered resources (registered resources / total expected resources)
(resources are executors in yarn mode and Kubernetes mode, CPU cores in standalone mode and Mesos coarse-grained
mode ['spark.cores.max' value is total expected resources for Mesos coarse-grained mode] )
to wait for before scheduling begins. Specified as a double between 0.0 and 1.0.
Regardless of whether the minimum ratio of resources has been reached,
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
the maximum amount of time it will wait before scheduling begins is controlled by config
<code>spark.scheduler.maxRegisteredResourcesWaitingTime</code>.
[SPARK-1946] Submit tasks after (configured ratio) executors have been registered Because submitting tasks and registering executors are asynchronous, in most situation, early stages' tasks run without preferred locality. A simple solution is sleeping few seconds in application, so that executors have enough time to register. The PR add 2 configuration properties to make TaskScheduler submit tasks after a few of executors have been registered. \# Submit tasks only after (registered executors / total executors) arrived the ratio, default value is 0 spark.scheduler.minRegisteredExecutorsRatio = 0.8 \# Whatever minRegisteredExecutorsRatio is arrived, submit tasks after the maxRegisteredWaitingTime(millisecond), default value is 30000 spark.scheduler.maxRegisteredExecutorsWaitingTime = 5000 Author: li-zhihui <zhihui.li@intel.com> Closes #900 from li-zhihui/master and squashes the following commits: b9f8326 [li-zhihui] Add logs & edit docs 1ac08b1 [li-zhihui] Add new configs to user docs 22ead12 [li-zhihui] Move waitBackendReady to postStartHook c6f0522 [li-zhihui] Bug fix: numExecutors wasn't set & use constant DEFAULT_NUMBER_EXECUTORS 4d6d847 [li-zhihui] Move waitBackendReady to TaskSchedulerImpl.start & some code refactor 0ecee9a [li-zhihui] Move waitBackendReady from DAGScheduler.submitStage to TaskSchedulerImpl.submitTasks 4261454 [li-zhihui] Add docs for new configs & code style ce0868a [li-zhihui] Code style, rename configuration property name of minRegisteredRatio & maxRegisteredWaitingTime 6cfb9ec [li-zhihui] Code style, revert default minRegisteredRatio of yarn to 0, driver get --num-executors in yarn/alpha 812c33c [li-zhihui] Fix driver lost --num-executors option in yarn-cluster mode e7b6272 [li-zhihui] support yarn-cluster 37f7dc2 [li-zhihui] support yarn mode(percentage style) 3f8c941 [li-zhihui] submit stage after (configured ratio of) executors have been registered
2014-07-14 16:32:49 -04:00
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>1.1.1</td>
[SPARK-1946] Submit tasks after (configured ratio) executors have been registered Because submitting tasks and registering executors are asynchronous, in most situation, early stages' tasks run without preferred locality. A simple solution is sleeping few seconds in application, so that executors have enough time to register. The PR add 2 configuration properties to make TaskScheduler submit tasks after a few of executors have been registered. \# Submit tasks only after (registered executors / total executors) arrived the ratio, default value is 0 spark.scheduler.minRegisteredExecutorsRatio = 0.8 \# Whatever minRegisteredExecutorsRatio is arrived, submit tasks after the maxRegisteredWaitingTime(millisecond), default value is 30000 spark.scheduler.maxRegisteredExecutorsWaitingTime = 5000 Author: li-zhihui <zhihui.li@intel.com> Closes #900 from li-zhihui/master and squashes the following commits: b9f8326 [li-zhihui] Add logs & edit docs 1ac08b1 [li-zhihui] Add new configs to user docs 22ead12 [li-zhihui] Move waitBackendReady to postStartHook c6f0522 [li-zhihui] Bug fix: numExecutors wasn't set & use constant DEFAULT_NUMBER_EXECUTORS 4d6d847 [li-zhihui] Move waitBackendReady to TaskSchedulerImpl.start & some code refactor 0ecee9a [li-zhihui] Move waitBackendReady from DAGScheduler.submitStage to TaskSchedulerImpl.submitTasks 4261454 [li-zhihui] Add docs for new configs & code style ce0868a [li-zhihui] Code style, rename configuration property name of minRegisteredRatio & maxRegisteredWaitingTime 6cfb9ec [li-zhihui] Code style, revert default minRegisteredRatio of yarn to 0, driver get --num-executors in yarn/alpha 812c33c [li-zhihui] Fix driver lost --num-executors option in yarn-cluster mode e7b6272 [li-zhihui] support yarn-cluster 37f7dc2 [li-zhihui] support yarn mode(percentage style) 3f8c941 [li-zhihui] submit stage after (configured ratio of) executors have been registered
2014-07-14 16:32:49 -04:00
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.scheduler.mode</code></td>
<td>FIFO</td>
[SPARK-1946] Submit tasks after (configured ratio) executors have been registered Because submitting tasks and registering executors are asynchronous, in most situation, early stages' tasks run without preferred locality. A simple solution is sleeping few seconds in application, so that executors have enough time to register. The PR add 2 configuration properties to make TaskScheduler submit tasks after a few of executors have been registered. \# Submit tasks only after (registered executors / total executors) arrived the ratio, default value is 0 spark.scheduler.minRegisteredExecutorsRatio = 0.8 \# Whatever minRegisteredExecutorsRatio is arrived, submit tasks after the maxRegisteredWaitingTime(millisecond), default value is 30000 spark.scheduler.maxRegisteredExecutorsWaitingTime = 5000 Author: li-zhihui <zhihui.li@intel.com> Closes #900 from li-zhihui/master and squashes the following commits: b9f8326 [li-zhihui] Add logs & edit docs 1ac08b1 [li-zhihui] Add new configs to user docs 22ead12 [li-zhihui] Move waitBackendReady to postStartHook c6f0522 [li-zhihui] Bug fix: numExecutors wasn't set & use constant DEFAULT_NUMBER_EXECUTORS 4d6d847 [li-zhihui] Move waitBackendReady to TaskSchedulerImpl.start & some code refactor 0ecee9a [li-zhihui] Move waitBackendReady from DAGScheduler.submitStage to TaskSchedulerImpl.submitTasks 4261454 [li-zhihui] Add docs for new configs & code style ce0868a [li-zhihui] Code style, rename configuration property name of minRegisteredRatio & maxRegisteredWaitingTime 6cfb9ec [li-zhihui] Code style, revert default minRegisteredRatio of yarn to 0, driver get --num-executors in yarn/alpha 812c33c [li-zhihui] Fix driver lost --num-executors option in yarn-cluster mode e7b6272 [li-zhihui] support yarn-cluster 37f7dc2 [li-zhihui] support yarn mode(percentage style) 3f8c941 [li-zhihui] submit stage after (configured ratio of) executors have been registered
2014-07-14 16:32:49 -04:00
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.8.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<tr>
<td><code>spark.scheduler.revive.interval</code></td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
<td>1s</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
The interval length for the scheduler to revive the worker resource offers to run tasks.
[SPARK-1946] Submit tasks after (configured ratio) executors have been registered Because submitting tasks and registering executors are asynchronous, in most situation, early stages' tasks run without preferred locality. A simple solution is sleeping few seconds in application, so that executors have enough time to register. The PR add 2 configuration properties to make TaskScheduler submit tasks after a few of executors have been registered. \# Submit tasks only after (registered executors / total executors) arrived the ratio, default value is 0 spark.scheduler.minRegisteredExecutorsRatio = 0.8 \# Whatever minRegisteredExecutorsRatio is arrived, submit tasks after the maxRegisteredWaitingTime(millisecond), default value is 30000 spark.scheduler.maxRegisteredExecutorsWaitingTime = 5000 Author: li-zhihui <zhihui.li@intel.com> Closes #900 from li-zhihui/master and squashes the following commits: b9f8326 [li-zhihui] Add logs & edit docs 1ac08b1 [li-zhihui] Add new configs to user docs 22ead12 [li-zhihui] Move waitBackendReady to postStartHook c6f0522 [li-zhihui] Bug fix: numExecutors wasn't set & use constant DEFAULT_NUMBER_EXECUTORS 4d6d847 [li-zhihui] Move waitBackendReady to TaskSchedulerImpl.start & some code refactor 0ecee9a [li-zhihui] Move waitBackendReady from DAGScheduler.submitStage to TaskSchedulerImpl.submitTasks 4261454 [li-zhihui] Add docs for new configs & code style ce0868a [li-zhihui] Code style, rename configuration property name of minRegisteredRatio & maxRegisteredWaitingTime 6cfb9ec [li-zhihui] Code style, revert default minRegisteredRatio of yarn to 0, driver get --num-executors in yarn/alpha 812c33c [li-zhihui] Fix driver lost --num-executors option in yarn-cluster mode e7b6272 [li-zhihui] support yarn-cluster 37f7dc2 [li-zhihui] support yarn mode(percentage style) 3f8c941 [li-zhihui] submit stage after (configured ratio of) executors have been registered
2014-07-14 16:32:49 -04:00
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.8.1</td>
[SPARK-1946] Submit tasks after (configured ratio) executors have been registered Because submitting tasks and registering executors are asynchronous, in most situation, early stages' tasks run without preferred locality. A simple solution is sleeping few seconds in application, so that executors have enough time to register. The PR add 2 configuration properties to make TaskScheduler submit tasks after a few of executors have been registered. \# Submit tasks only after (registered executors / total executors) arrived the ratio, default value is 0 spark.scheduler.minRegisteredExecutorsRatio = 0.8 \# Whatever minRegisteredExecutorsRatio is arrived, submit tasks after the maxRegisteredWaitingTime(millisecond), default value is 30000 spark.scheduler.maxRegisteredExecutorsWaitingTime = 5000 Author: li-zhihui <zhihui.li@intel.com> Closes #900 from li-zhihui/master and squashes the following commits: b9f8326 [li-zhihui] Add logs & edit docs 1ac08b1 [li-zhihui] Add new configs to user docs 22ead12 [li-zhihui] Move waitBackendReady to postStartHook c6f0522 [li-zhihui] Bug fix: numExecutors wasn't set & use constant DEFAULT_NUMBER_EXECUTORS 4d6d847 [li-zhihui] Move waitBackendReady to TaskSchedulerImpl.start & some code refactor 0ecee9a [li-zhihui] Move waitBackendReady from DAGScheduler.submitStage to TaskSchedulerImpl.submitTasks 4261454 [li-zhihui] Add docs for new configs & code style ce0868a [li-zhihui] Code style, rename configuration property name of minRegisteredRatio & maxRegisteredWaitingTime 6cfb9ec [li-zhihui] Code style, revert default minRegisteredRatio of yarn to 0, driver get --num-executors in yarn/alpha 812c33c [li-zhihui] Fix driver lost --num-executors option in yarn-cluster mode e7b6272 [li-zhihui] support yarn-cluster 37f7dc2 [li-zhihui] support yarn mode(percentage style) 3f8c941 [li-zhihui] submit stage after (configured ratio of) executors have been registered
2014-07-14 16:32:49 -04:00
</tr>
<tr>
<td><code>spark.scheduler.listenerbus.eventqueue.capacity</code></td>
<td>10000</td>
<td>
The default capacity for event queues. Spark will try to initialize an event queue
using capacity specified by `spark.scheduler.listenerbus.eventqueue.queueName.capacity`
first. If it's not configured, Spark will use the default capacity specified by this
config. Note that capacity must be greater than 0. Consider increasing value (e.g. 20000)
if listener events are dropped. Increasing this value may result in the driver using more memory.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.3.0</td>
</tr>
<tr>
<td><code>spark.scheduler.listenerbus.eventqueue.shared.capacity</code></td>
<td><code>spark.scheduler.listenerbus.eventqueue.capacity</code></td>
<td>
Capacity for shared event queue in Spark listener bus, which hold events for external listener(s)
that register to the listener bus. Consider increasing value, if the listener events corresponding
to shared queue are dropped. Increasing this value may result in the driver using more memory.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.scheduler.listenerbus.eventqueue.appStatus.capacity</code></td>
<td><code>spark.scheduler.listenerbus.eventqueue.capacity</code></td>
<td>
Capacity for appStatus event queue, which hold events for internal application status listeners.
Consider increasing value, if the listener events corresponding to appStatus queue are dropped.
Increasing this value may result in the driver using more memory.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.scheduler.listenerbus.eventqueue.executorManagement.capacity</code></td>
<td><code>spark.scheduler.listenerbus.eventqueue.capacity</code></td>
<td>
Capacity for executorManagement event queue in Spark listener bus, which hold events for internal
executor management listeners. Consider increasing value if the listener events corresponding to
executorManagement queue are dropped. Increasing this value may result in the driver using more memory.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.scheduler.listenerbus.eventqueue.eventLog.capacity</code></td>
<td><code>spark.scheduler.listenerbus.eventqueue.capacity</code></td>
<td>
Capacity for eventLog queue in Spark listener bus, which hold events for Event logging listeners
that write events to eventLogs. Consider increasing value if the listener events corresponding to eventLog queue
are dropped. Increasing this value may result in the driver using more memory.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.scheduler.listenerbus.eventqueue.streams.capacity</code></td>
<td><code>spark.scheduler.listenerbus.eventqueue.capacity</code></td>
<td>
Capacity for streams queue in Spark listener bus, which hold events for internal streaming listener.
Consider increasing value if the listener events corresponding to streams queue are dropped. Increasing
this value may result in the driver using more memory.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.scheduler.blacklist.unschedulableTaskSetTimeout</code></td>
<td>120s</td>
<td>
The timeout in seconds to wait to acquire a new executor and schedule a task before aborting a
TaskSet which is unschedulable because of being completely blacklisted.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>2.4.1</td>
</tr>
<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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.1.0</td>
</tr>
<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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.1.0</td>
</tr>
<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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.1.0</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.1.0</td>
</tr>
<tr>
<td><code>spark.blacklist.stage.maxFailedTasksPerExecutor</code></td>
<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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.1.0</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.1.0</td>
</tr>
<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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.2.0</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.2.0</td>
</tr>
<tr>
<td><code>spark.blacklist.killBlacklistedExecutors</code></td>
<td>false</td>
<td>
(Experimental) If set to "true", allow Spark to automatically kill the executors
when they are blacklisted on fetch failure or blacklisted for the entire application,
as controlled by spark.blacklist.application.*. Note that, when an entire node is added
to the blacklist, all of the executors on that node will be killed.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.2.0</td>
</tr>
[SPARK-13669][SPARK-20898][CORE] Improve the blacklist mechanism to handle external shuffle service unavailable situation ## What changes were proposed in this pull request? Currently we are running into an issue with Yarn work preserving enabled + external shuffle service. In the work preserving enabled scenario, the failure of NM will not lead to the exit of executors, so executors can still accept and run the tasks. The problem here is when NM is failed, external shuffle service is actually inaccessible, so reduce tasks will always complain about the “Fetch failure”, and the failure of reduce stage will make the parent stage (map stage) rerun. The tricky thing here is Spark scheduler is not aware of the unavailability of external shuffle service, and will reschedule the map tasks on the executor where NM is failed, and again reduce stage will be failed with “Fetch failure”, and after 4 retries, the job is failed. This could also apply to other cluster manager with external shuffle service. So here the main problem is that we should avoid assigning tasks to those bad executors (where shuffle service is unavailable). Current Spark's blacklist mechanism could blacklist executors/nodes by failure tasks, but it doesn't handle this specific fetch failure scenario. So here propose to improve the current application blacklist mechanism to handle fetch failure issue (especially with external shuffle service unavailable issue), to blacklist the executors/nodes where shuffle fetch is unavailable. ## How was this patch tested? Unit test and small cluster verification. Author: jerryshao <sshao@hortonworks.com> Closes #17113 from jerryshao/SPARK-13669.
2017-06-26 12:14:03 -04:00
<tr>
<td><code>spark.blacklist.application.fetchFailure.enabled</code></td>
<td>false</td>
<td>
(Experimental) If set to "true", Spark will blacklist the executor immediately when a fetch
failure happens. If external shuffle service is enabled, then the whole node will be
[SPARK-13669][SPARK-20898][CORE] Improve the blacklist mechanism to handle external shuffle service unavailable situation ## What changes were proposed in this pull request? Currently we are running into an issue with Yarn work preserving enabled + external shuffle service. In the work preserving enabled scenario, the failure of NM will not lead to the exit of executors, so executors can still accept and run the tasks. The problem here is when NM is failed, external shuffle service is actually inaccessible, so reduce tasks will always complain about the “Fetch failure”, and the failure of reduce stage will make the parent stage (map stage) rerun. The tricky thing here is Spark scheduler is not aware of the unavailability of external shuffle service, and will reschedule the map tasks on the executor where NM is failed, and again reduce stage will be failed with “Fetch failure”, and after 4 retries, the job is failed. This could also apply to other cluster manager with external shuffle service. So here the main problem is that we should avoid assigning tasks to those bad executors (where shuffle service is unavailable). Current Spark's blacklist mechanism could blacklist executors/nodes by failure tasks, but it doesn't handle this specific fetch failure scenario. So here propose to improve the current application blacklist mechanism to handle fetch failure issue (especially with external shuffle service unavailable issue), to blacklist the executors/nodes where shuffle fetch is unavailable. ## How was this patch tested? Unit test and small cluster verification. Author: jerryshao <sshao@hortonworks.com> Closes #17113 from jerryshao/SPARK-13669.
2017-06-26 12:14:03 -04:00
blacklisted.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.3.0</td>
[SPARK-13669][SPARK-20898][CORE] Improve the blacklist mechanism to handle external shuffle service unavailable situation ## What changes were proposed in this pull request? Currently we are running into an issue with Yarn work preserving enabled + external shuffle service. In the work preserving enabled scenario, the failure of NM will not lead to the exit of executors, so executors can still accept and run the tasks. The problem here is when NM is failed, external shuffle service is actually inaccessible, so reduce tasks will always complain about the “Fetch failure”, and the failure of reduce stage will make the parent stage (map stage) rerun. The tricky thing here is Spark scheduler is not aware of the unavailability of external shuffle service, and will reschedule the map tasks on the executor where NM is failed, and again reduce stage will be failed with “Fetch failure”, and after 4 retries, the job is failed. This could also apply to other cluster manager with external shuffle service. So here the main problem is that we should avoid assigning tasks to those bad executors (where shuffle service is unavailable). Current Spark's blacklist mechanism could blacklist executors/nodes by failure tasks, but it doesn't handle this specific fetch failure scenario. So here propose to improve the current application blacklist mechanism to handle fetch failure issue (especially with external shuffle service unavailable issue), to blacklist the executors/nodes where shuffle fetch is unavailable. ## How was this patch tested? Unit test and small cluster verification. Author: jerryshao <sshao@hortonworks.com> Closes #17113 from jerryshao/SPARK-13669.
2017-06-26 12:14:03 -04:00
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.speculation</code></td>
<td>false</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.6.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<tr>
<td><code>spark.speculation.interval</code></td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
<td>100ms</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
How often Spark will check for tasks to speculate.
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.6.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.6.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<tr>
<td><code>spark.speculation.quantile</code></td>
<td>0.75</td>
<td>
Fraction of tasks which must be complete before speculation is enabled for a particular stage.
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>0.6.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
<tr>
<td><code>spark.speculation.task.duration.threshold</code></td>
<td>None</td>
<td>
Task duration after which scheduler would try to speculative run the task. If provided, tasks
would be speculatively run if current stage contains less tasks than or equal to the number of
slots on a single executor and the task is taking longer time than the threshold. This config
helps speculate stage with very few tasks. Regular speculation configs may also apply if the
executor slots are large enough. E.g. tasks might be re-launched if there are enough successful
runs even though the threshold hasn't been reached. The number of slots is computed based on
the conf values of spark.executor.cores and spark.task.cpus minimum 1.
Default unit is bytes, unless otherwise specified.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>3.0.0</td>
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<tr>
<td><code>spark.task.cpus</code></td>
<td>1</td>
<td>
Number of cores to allocate for each task.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>0.5.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
<tr>
<td><code>spark.task.resource.{resourceName}.amount</code></td>
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
<td>1</td>
<td>
Amount of a particular resource type to allocate for each task, note that this can be a double.
If this is specified you must also provide the executor config
<code>spark.executor.resource.{resourceName}.amount</code> and any corresponding discovery configs
so that your executors are created with that resource type. In addition to whole amounts,
a fractional amount (for example, 0.25, which means 1/4th of a resource) may be specified.
Fractional amounts must be less than or equal to 0.5, or in other words, the minimum amount of
resource sharing is 2 tasks per resource. Additionally, fractional amounts are floored
in order to assign resource slots (e.g. a 0.2222 configuration, or 1/0.2222 slots will become
4 tasks/resource, not 5).
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
[SPARK-27024] Executor interface for cluster managers to support GPU and other resources ## What changes were proposed in this pull request? Add in GPU and generic resource type allocation to the executors. Note this is part of a bigger feature for gpu-aware scheduling and is just how the executor find the resources. The general flow : - users ask for a certain set of resources, for instance number of gpus - each cluster manager has a specific way to do this. - cluster manager allocates a container or set of resources (standalone mode) - When spark launches the executor in that container, the executor either has to be told what resources it has or it has to auto discover them. - Executor has to register with Driver and tell the driver the set of resources it has so the scheduler can use that to schedule tasks that requires a certain amount of each of those resources In this pr I added configs and arguments to the executor to be able discover resources. The argument to the executor is intended to be used by standalone mode or other cluster managers that don't have isolation so that it can assign specific resources to specific executors in case there are multiple executors on a node. The argument is a file contains JSON Array of ResourceInformation objects. The discovery script is meant to be used in an isolated environment where the executor only sees the resources it should use. Note that there will be follow on PRs to add other parts like the scheduler part. See the epic high level jira: https://issues.apache.org/jira/browse/SPARK-24615 ## How was this patch tested? Added unit tests and manually tested. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24406 from tgravescs/gpu-sched-executor-clean. Authored-by: Thomas Graves <tgraves@nvidia.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-05-14 09:41:41 -04:00
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<tr>
<td><code>spark.task.maxFailures</code></td>
<td>4</td>
<td>
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.
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
Should be greater than or equal to 1. Number of allowed retries = this value - 1.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>0.8.0</td>
</tr>
[SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors ## What changes were proposed in this pull request? Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks. This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks. This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details. ## How was this patch tested? Tested via a new test case in `JobCancellationSuite`, plus manual testing. Author: Josh Rosen <joshrosen@databricks.com> Closes #16189 from JoshRosen/cancellation.
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
of this monitoring. When set to false (the default), task killing will use an older code
[SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors ## What changes were proposed in this pull request? Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks. This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks. This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details. ## How was this patch tested? Tested via a new test case in `JobCancellationSuite`, plus manual testing. Author: Josh Rosen <joshrosen@databricks.com> Closes #16189 from JoshRosen/cancellation.
2016-12-19 21:43:59 -05:00
path which lacks such monitoring.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.0.3</td>
[SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors ## What changes were proposed in this pull request? Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks. This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks. This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details. ## How was this patch tested? Tested via a new test case in `JobCancellationSuite`, plus manual testing. Author: Josh Rosen <joshrosen@databricks.com> Closes #16189 from JoshRosen/cancellation.
2016-12-19 21:43:59 -05:00
</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.0.3</td>
[SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors ## What changes were proposed in this pull request? Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks. This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks. This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details. ## How was this patch tested? Tested via a new test case in `JobCancellationSuite`, plus manual testing. Author: Josh Rosen <joshrosen@databricks.com> Closes #16189 from JoshRosen/cancellation.
2016-12-19 21:43:59 -05:00
</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.0.3</td>
[SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors ## What changes were proposed in this pull request? Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks. This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks. This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details. ## How was this patch tested? Tested via a new test case in `JobCancellationSuite`, plus manual testing. Author: Josh Rosen <joshrosen@databricks.com> Closes #16189 from JoshRosen/cancellation.
2016-12-19 21:43:59 -05:00
</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 noncancellable tasks from rendering
[SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors ## What changes were proposed in this pull request? Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks. This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks. This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details. ## How was this patch tested? Tested via a new test case in `JobCancellationSuite`, plus manual testing. Author: Josh Rosen <joshrosen@databricks.com> Closes #16189 from JoshRosen/cancellation.
2016-12-19 21:43:59 -05:00
an executor unusable.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.0.3</td>
</tr>
<tr>
<td><code>spark.stage.maxConsecutiveAttempts</code></td>
<td>4</td>
<td>
Number of consecutive stage attempts allowed before a stage is aborted.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>2.2.0</td>
[SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors ## What changes were proposed in this pull request? Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks. This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks. This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details. ## How was this patch tested? Tested via a new test case in `JobCancellationSuite`, plus manual testing. Author: Josh Rosen <joshrosen@databricks.com> Closes #16189 from JoshRosen/cancellation.
2016-12-19 21:43:59 -05:00
</tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</table>
### Barrier Execution Mode
<table class="table">
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td><code>spark.barrier.sync.timeout</code></td>
<td>365d</td>
<td>
The timeout in seconds for each <code>barrier()</code> call from a barrier task. If the
coordinator didn't receive all the sync messages from barrier tasks within the
configured time, throw a SparkException to fail all the tasks. The default value is set
to 31536000(3600 * 24 * 365) so the <code>barrier()</code> call shall wait for one year.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>2.4.0</td>
</tr>
<tr>
<td><code>spark.scheduler.barrier.maxConcurrentTasksCheck.interval</code></td>
<td>15s</td>
<td>
Time in seconds to wait between a max concurrent tasks check failure and the next
check. A max concurrent tasks check ensures the cluster can launch more concurrent
tasks than required by a barrier stage on job submitted. The check can fail in case
a cluster has just started and not enough executors have registered, so we wait for a
little while and try to perform the check again. If the check fails more than a
configured max failure times for a job then fail current job submission. Note this
config only applies to jobs that contain one or more barrier stages, we won't perform
the check on non-barrier jobs.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>2.4.0</td>
</tr>
<tr>
<td><code>spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures</code></td>
<td>40</td>
<td>
Number of max concurrent tasks check failures allowed before fail a job submission.
A max concurrent tasks check ensures the cluster can launch more concurrent tasks than
required by a barrier stage on job submitted. The check can fail in case a cluster
has just started and not enough executors have registered, so we wait for a little
while and try to perform the check again. If the check fails more than a configured
max failure times for a job then fail current job submission. Note this config only
applies to jobs that contain one or more barrier stages, we won't perform the check on
non-barrier jobs.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<td>2.4.0</td>
</tr>
</table>
### Dynamic Allocation
<table class="table">
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up #27847, #27852 and https://github.com/apache/spark/pull/27913. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.storage.localDiskByExecutors.cacheSize | 3.0.0 | SPARK-27651 | fd2bf55abaab08798a428d4e47d4050ba2b82a95#diff-6bdad48cfc34314e89599655442ff210 | spark.storage.memoryMapLimitForTests | 2.3.0 | SPARK-3151 | b8ffb51055108fd606b86f034747006962cd2df3#diff-abd96f2ae793cd6ea6aab5b96a3c1d7a |   spark.barrier.sync.timeout | 2.4.0 | SPARK-24817 | 388f5a0635a2812cd71b08352e3ddc20293ec189#diff-6bdad48cfc34314e89599655442ff210 | spark.scheduler.blacklist.unschedulableTaskSetTimeout | 2.4.1 | SPARK-22148 | 52e9711d01694158ecb3691f2ec25c0ebe4b0207#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.interval | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures | 2.4.0 | SPARK-24819 | bfb74394a5513134ea1da9fcf4a1783b77dd64e4#diff-6bdad48cfc34314e89599655442ff210 |   spark.unsafe.exceptionOnMemoryLeak | 1.4.0 | SPARK-7076 and SPARK-7077 and SPARK-7080 | f49284b5bf3a69ed91a5e3e6e0ed3be93a6ab9e4#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.unsafe.sorter.spill.read.ahead.enabled | 2.3.0 | SPARK-21113 | 1e978b17d63d7ba20368057aa4e65f5ef6e87369#diff-93a086317cea72a113cf81056882c206 |   spark.unsafe.sorter.spill.reader.buffer.size | 2.1.0 | SPARK-16862 | c1937dd19a23bd096a4707656c7ba19fb5c16966#diff-93a086317cea72a113cf81056882c206 |   spark.plugins | 3.0.0 | SPARK-29397 | d51d228048d519a9a666f48dc532625de13e7587#diff-6bdad48cfc34314e89599655442ff210 |   spark.cleaner.periodicGC.interval | 1.6.0 | SPARK-8414 | 72da2a21f0940b97757ace5975535e559d627688#diff-75141521b1d55bc32d72b70032ad96c0 | spark.cleaner.referenceTracking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking | 1.0.0 | SPARK-1103 | 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d#diff-364713d7776956cb8b0a771e9b62f82d |   spark.cleaner.referenceTracking.blocking.shuffle | 1.1.1 | SPARK-3139 | 5cf1e440137006eedd6846ac8fa57ccf9fd1958d#diff-75141521b1d55bc32d72b70032ad96c0 |   spark.cleaner.referenceTracking.cleanCheckpoints | 1.4.0 | SPARK-2033 | 25998e4d73bcc95ac85d9af71adfdc726ec89568#diff-440e866c5df0b8386aff57f9f8bd8db1 |   spark.executor.logs.rolling.strategy | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.time.interval | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.maxSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.executor.logs.rolling.maxRetainedFiles | 1.1.0 | SPARK-1940 | 4823bf470ec1b47a6f404834d4453e61d3dcbec9#diff-2b4575e096e4db7165e087f9429f2a02 | spark.executor.logs.rolling.enableCompression | 2.0.2 | SPARK-17711 | 26e978a93f029e1a1b5c7524d0b52c8141b70997#diff-2b4575e096e4db7165e087f9429f2a02 |   spark.master.rest.enabled | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.rest.port | 1.3.0 | SPARK-5388 | 6ec0cdc14390d4dc45acf31040f21e1efc476fc0#diff-29dffdccd5a7f4c8b496c293e87c8668 |   spark.master.ui.port | 1.1.0 | SPARK-2857 | 12f99cf5f88faf94d9dbfe85cb72d0010a3a25ac#diff-366c88f47e9b5cfa4d4305febeb8b026 |   spark.io.compression.snappy.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.lz4.blockSize | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.io.compression.codec | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.bufferSize | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.compression.zstd.level | 2.3.0 | SPARK-19112 | 444bce1c98c45147fe63e2132e9743a0c5e49598#diff-df9e6118c481ceb27faa399114fac0a1 |   spark.io.warning.largeFileThreshold | 3.0.0 | SPARK-28366 | 26d03b62e20d053943d03b5c5573dd349e49654c#diff-6bdad48cfc34314e89599655442ff210 |   spark.eventLog.compression.codec | 3.0.0 | SPARK-28118 | 47f54b1ec717d0d744bf3ad46bb1ed3542b667c8#diff-6bdad48cfc34314e89599655442ff210 |   spark.buffer.size | 0.5.0 | None | 4b1646a25f7581cecae108553da13833e842e68a#diff-eaf125f56ce786d64dcef99cf446a751 |   spark.locality.wait.process | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.node | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.locality.wait.rack | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.reducer.maxSizeInFlight | 1.4.0 | SPARK-5932 | 2d222fb39dd978e5a33cde6ceb59307cbdf7b171#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.reducer.maxReqsInFlight | 2.0.0 | SPARK-6166 | 894921d813a259f2f266fde7d86d2ecb5a0af24b#diff-eb30a71e0d04150b8e0b64929852e38b |   spark.broadcast.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.broadcast.blockSize | 0.5.0 | None | b8ab7862b8bd168bca60bd930cd97c1099fbc8a8#diff-271d7958e14cdaa46cf3737cfcf51341 |   spark.broadcast.checksum | 2.1.1 | SPARK-18188 | 06a56df226aa0c03c21f23258630d8a96385c696#diff-4f43d14923008c6650a8eb7b40c07f74 | spark.broadcast.UDFCompressionThreshold | 3.0.0 | SPARK-28355 | 79e204770300dab4a669b9f8e2421ef905236e7b#diff-6bdad48cfc34314e89599655442ff210 | spark.rdd.compress | 0.6.0 | None | efc5423210d1aadeaea78273a4a8f10425753079#diff-76170a9c8f67b542bc58240a0a12fe08 |   spark.rdd.parallelListingThreshold | 2.0.0 | SPARK-9926 | 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a#diff-eaababfc87ea4949f97860e8b89b7586 | spark.rdd.limit.scaleUpFactor | 2.1.0 | SPARK-16984 | 806d8a8e980d8ba2f4261bceb393c40bafaa2f73#diff-1d55e54678eff2076263f2fe36150c17 |   spark.serializer | 0.5.0 | None | fd1d255821bde844af28e897fabd59a715659038#diff-b920b65c23bf3a1b3326325b0d6a81b2 |   spark.serializer.objectStreamReset | 1.0.0 | SPARK-942 | 40566e10aae4b21ffc71ea72702b8df118ac5c8e#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.serializer.extraDebugInfo | 1.3.0 | SPARK-5307 | 636408311deeebd77fb83d2249e0afad1a1ba149#diff-6a59dfc43d1b31dc1c3072ceafa829f5 |   spark.jars | 0.9.0 | None | f1d206c6b4c0a5b2517b05af05fdda6049e2f7c2#diff-364713d7776956cb8b0a771e9b62f82d |   spark.files | 1.0.0 | None | 29ee101c73bf066bf7f4f8141c475b8d1bd3cf1c#diff-364713d7776956cb8b0a771e9b62f82d |   spark.submit.deployMode | 1.5.0 | SPARK-6797 | 7f487c8bde14dbdd244a3493ad11a129ef2bb327#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.submit.pyFiles | 1.0.1 | SPARK-1549 | d7ddb26e1fa02e773999cc4a97c48d2cd1723956#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.scheduler.allocation.file | 0.8.1 | None | 976fe60f7609d7b905a34f18743efabd966407f0#diff-9bc0105ee454005379abed710cd20ced |   spark.scheduler.minRegisteredResourcesRatio | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.maxRegisteredResourcesWaitingTime | 1.1.1 | SPARK-2635 | 3311da2f9efc5ff2c7d01273ac08f719b067d11d#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.scheduler.mode | 0.8.0 | None | 98fb69822cf780160bca51abeaab7c82e49fab54#diff-cb7a25b3c9a7341c6d99bcb8e9780c92 |   spark.scheduler.revive.interval | 0.8.1 | None | d0c9d41a061969d409715b86a91937d8de4c29f7#diff-7d99a7c7a051e5e851aaaefb275a44a1 |   spark.speculation | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.interval | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-4e188f32951dc989d97fa7577858bc7c |   spark.speculation.multiplier | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.quantile | 0.6.0 | None | e72afdb817bcc8388aeb8b8d31628fd5fd67acf1#diff-fff59f72dfe6ca4ccb607ad12535da07 |   spark.speculation.task.duration.threshold | 3.0.0 | SPARK-29976 | ad238a2238a9d0da89be4424574436cbfaee579d#diff-6bdad48cfc34314e89599655442ff210 | spark.yarn.stagingDir | 2.0.0 | SPARK-13063 | bc36df127d3b9f56b4edaeb5eca7697d4aef761a#diff-14b8ed2ef4e3da985300b8d796a38fa9 |   spark.buffer.pageSize | 1.5.0 | SPARK-9411 | 1b0099fc62d02ff6216a76fbfe17a4ec5b2f3536#diff-1b22e54318c04824a6d53ed3f4d1bb35 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27931 from beliefer/add-version-to-core-config-part-four. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-22 22:07:43 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td><code>spark.dynamicAllocation.enabled</code></td>
<td>false</td>
<td>
Whether to use dynamic resource allocation, which scales the number of executors registered
with this application up and down based on the workload.
For more detail, see the description
<a href="job-scheduling.html#dynamic-resource-allocation">here</a>.
<br><br>
This requires <code>spark.shuffle.service.enabled</code> to be set.
The following configurations are also relevant:
<code>spark.dynamicAllocation.minExecutors</code>,
<code>spark.dynamicAllocation.maxExecutors</code>, and
<code>spark.dynamicAllocation.initialExecutors</code>
<code>spark.dynamicAllocation.executorAllocationRatio</code>
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>1.2.0</td>
</tr>
<tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td><code>spark.dynamicAllocation.executorIdleTimeout</code></td>
<td>60s</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td>
If dynamic allocation is enabled and an executor has been idle for more than this duration,
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
the executor will be removed. For more detail, see this
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<a href="job-scheduling.html#resource-allocation-policy">description</a>.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>1.2.0</td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
</tr>
[SPARK-7955] [CORE] Ensure executors with cached RDD blocks are not re… …moved if dynamic allocation is enabled. This is a work in progress. This patch ensures that an executor that has cached RDD blocks are not removed, but makes no attempt to find another executor to remove. This is meant to get some feedback on the current approach, and if it makes sense then I will look at choosing another executor to remove. No testing has been done either. Author: Hari Shreedharan <hshreedharan@apache.org> Closes #6508 from harishreedharan/dymanic-caching and squashes the following commits: dddf1eb [Hari Shreedharan] Minor configuration description update. 10130e2 [Hari Shreedharan] Fix compile issue. 5417b53 [Hari Shreedharan] Add documentation for new config. Remove block from cachedBlocks when it is dropped. 875916a [Hari Shreedharan] Make some code more readable. 39940ca [Hari Shreedharan] Handle the case where the executor has not yet registered. 90ad711 [Hari Shreedharan] Remove unused imports and unused methods. 063985c [Hari Shreedharan] Send correct message instead of recursively calling same method. ec2fd7e [Hari Shreedharan] Add file missed in last commit 5d10fad [Hari Shreedharan] Update cached blocks status using local info, rather than doing an RPC. 193af4c [Hari Shreedharan] WIP. Use local state rather than via RPC. ae932ff [Hari Shreedharan] Fix config param name. 272969d [Hari Shreedharan] Fix seconds to millis bug. 5a1993f [Hari Shreedharan] Add timeout for cache executors. Ignore broadcast blocks while checking if there are cached blocks. 57fefc2 [Hari Shreedharan] [SPARK-7955][Core] Ensure executors with cached RDD blocks are not removed if dynamic allocation is enabled.
2015-06-07 00:13:26 -04:00
<tr>
<td><code>spark.dynamicAllocation.cachedExecutorIdleTimeout</code></td>
<td>infinity</td>
[SPARK-7955] [CORE] Ensure executors with cached RDD blocks are not re… …moved if dynamic allocation is enabled. This is a work in progress. This patch ensures that an executor that has cached RDD blocks are not removed, but makes no attempt to find another executor to remove. This is meant to get some feedback on the current approach, and if it makes sense then I will look at choosing another executor to remove. No testing has been done either. Author: Hari Shreedharan <hshreedharan@apache.org> Closes #6508 from harishreedharan/dymanic-caching and squashes the following commits: dddf1eb [Hari Shreedharan] Minor configuration description update. 10130e2 [Hari Shreedharan] Fix compile issue. 5417b53 [Hari Shreedharan] Add documentation for new config. Remove block from cachedBlocks when it is dropped. 875916a [Hari Shreedharan] Make some code more readable. 39940ca [Hari Shreedharan] Handle the case where the executor has not yet registered. 90ad711 [Hari Shreedharan] Remove unused imports and unused methods. 063985c [Hari Shreedharan] Send correct message instead of recursively calling same method. ec2fd7e [Hari Shreedharan] Add file missed in last commit 5d10fad [Hari Shreedharan] Update cached blocks status using local info, rather than doing an RPC. 193af4c [Hari Shreedharan] WIP. Use local state rather than via RPC. ae932ff [Hari Shreedharan] Fix config param name. 272969d [Hari Shreedharan] Fix seconds to millis bug. 5a1993f [Hari Shreedharan] Add timeout for cache executors. Ignore broadcast blocks while checking if there are cached blocks. 57fefc2 [Hari Shreedharan] [SPARK-7955][Core] Ensure executors with cached RDD blocks are not removed if dynamic allocation is enabled.
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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>1.4.0</td>
[SPARK-7955] [CORE] Ensure executors with cached RDD blocks are not re… …moved if dynamic allocation is enabled. This is a work in progress. This patch ensures that an executor that has cached RDD blocks are not removed, but makes no attempt to find another executor to remove. This is meant to get some feedback on the current approach, and if it makes sense then I will look at choosing another executor to remove. No testing has been done either. Author: Hari Shreedharan <hshreedharan@apache.org> Closes #6508 from harishreedharan/dymanic-caching and squashes the following commits: dddf1eb [Hari Shreedharan] Minor configuration description update. 10130e2 [Hari Shreedharan] Fix compile issue. 5417b53 [Hari Shreedharan] Add documentation for new config. Remove block from cachedBlocks when it is dropped. 875916a [Hari Shreedharan] Make some code more readable. 39940ca [Hari Shreedharan] Handle the case where the executor has not yet registered. 90ad711 [Hari Shreedharan] Remove unused imports and unused methods. 063985c [Hari Shreedharan] Send correct message instead of recursively calling same method. ec2fd7e [Hari Shreedharan] Add file missed in last commit 5d10fad [Hari Shreedharan] Update cached blocks status using local info, rather than doing an RPC. 193af4c [Hari Shreedharan] WIP. Use local state rather than via RPC. ae932ff [Hari Shreedharan] Fix config param name. 272969d [Hari Shreedharan] Fix seconds to millis bug. 5a1993f [Hari Shreedharan] Add timeout for cache executors. Ignore broadcast blocks while checking if there are cached blocks. 57fefc2 [Hari Shreedharan] [SPARK-7955][Core] Ensure executors with cached RDD blocks are not removed if dynamic allocation is enabled.
2015-06-07 00:13:26 -04:00
</tr>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<tr>
<td><code>spark.dynamicAllocation.initialExecutors</code></td>
<td><code>spark.dynamicAllocation.minExecutors</code></td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
Initial number of executors to run if dynamic allocation is enabled.
<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.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>1.3.0</td>
</tr>
<tr>
<td><code>spark.dynamicAllocation.maxExecutors</code></td>
<td>infinity</td>
<td>
Upper bound for the number of executors if dynamic allocation is enabled.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>1.2.0</td>
</tr>
<tr>
<td><code>spark.dynamicAllocation.minExecutors</code></td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
<td>0</td>
<td>
[SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York <brennon.york@capitalone.com> Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties
2015-02-25 19:12:56 -05:00
Lower bound for the number of executors if dynamic allocation is enabled.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>1.2.0</td>
</tr>
<tr>
<td><code>spark.dynamicAllocation.executorAllocationRatio</code></td>
<td>1</td>
<td>
By default, the dynamic allocation will request enough executors to maximize the
parallelism according to the number of tasks to process. While this minimizes the
latency of the job, with small tasks this setting can waste a lot of resources due to
executor allocation overhead, as some executor might not even do any work.
This setting allows to set a ratio that will be used to reduce the number of
executors w.r.t. full parallelism.
Defaults to 1.0 to give maximum parallelism.
0.5 will divide the target number of executors by 2
Fix typos detected by github.com/client9/misspell ## What changes were proposed in this pull request? Fixing typos is sometimes very hard. It's not so easy to visually review them. Recently, I discovered a very useful tool for it, [misspell](https://github.com/client9/misspell). This pull request fixes minor typos detected by [misspell](https://github.com/client9/misspell) except for the false positives. If you would like me to work on other files as well, let me know. ## How was this patch tested? ### before ``` $ misspell . | grep -v '.js' R/pkg/R/SQLContext.R:354:43: "definiton" is a misspelling of "definition" R/pkg/R/SQLContext.R:424:43: "definiton" is a misspelling of "definition" R/pkg/R/SQLContext.R:445:43: "definiton" is a misspelling of "definition" R/pkg/R/SQLContext.R:495:43: "definiton" is a misspelling of "definition" NOTICE-binary:454:16: "containd" is a misspelling of "contained" R/pkg/R/context.R:46:43: "definiton" is a misspelling of "definition" R/pkg/R/context.R:74:43: "definiton" is a misspelling of "definition" R/pkg/R/DataFrame.R:591:48: "persistance" is a misspelling of "persistence" R/pkg/R/streaming.R:166:44: "occured" is a misspelling of "occurred" R/pkg/inst/worker/worker.R:65:22: "ouput" is a misspelling of "output" R/pkg/tests/fulltests/test_utils.R:106:25: "environemnt" is a misspelling of "environment" common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java:38:39: "existant" is a misspelling of "existent" common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java:83:39: "existant" is a misspelling of "existent" common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:243:46: "transfered" is a misspelling of "transferred" common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:234:19: "transfered" is a misspelling of "transferred" common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:238:63: "transfered" is a misspelling of "transferred" common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:244:46: "transfered" is a misspelling of "transferred" common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:276:39: "transfered" is a misspelling of "transferred" common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred" common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala:195:15: "orgin" is a misspelling of "origin" core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:621:39: "gauranteed" is a misspelling of "guaranteed" core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc" core/src/main/scala/org/apache/spark/storage/DiskStore.scala:282:18: "transfered" is a misspelling of "transferred" core/src/main/scala/org/apache/spark/util/ListenerBus.scala:64:17: "overriden" is a misspelling of "overridden" core/src/test/scala/org/apache/spark/ShuffleSuite.scala:211:7: "substracted" is a misspelling of "subtracted" core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture" core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:2468:84: "truely" is a misspelling of "truly" core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:25:18: "persistance" is a misspelling of "persistence" core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:26:69: "persistance" is a misspelling of "persistence" data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous" dev/run-pip-tests:55:28: "enviroments" is a misspelling of "environments" dev/run-pip-tests:91:37: "virutal" is a misspelling of "virtual" dev/merge_spark_pr.py:377:72: "accross" is a misspelling of "across" dev/merge_spark_pr.py:378:66: "accross" is a misspelling of "across" dev/run-pip-tests:126:25: "enviroments" is a misspelling of "environments" docs/configuration.md:1830:82: "overriden" is a misspelling of "overridden" docs/structured-streaming-programming-guide.md:525:45: "processs" is a misspelling of "processes" docs/structured-streaming-programming-guide.md:1165:61: "BETWEN" is a misspelling of "BETWEEN" docs/sql-programming-guide.md:1891:810: "behaivor" is a misspelling of "behavior" examples/src/main/python/sql/arrow.py:98:8: "substract" is a misspelling of "subtract" examples/src/main/python/sql/arrow.py:103:27: "substract" is a misspelling of "subtract" licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching" licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics" licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching" licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics" licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching" licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics" licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING" licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS" licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING" licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS" licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching" licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics" licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching" licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics" licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching" licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics" licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING" licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS" licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING" licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS" mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the" mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels" mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean" mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean" mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The" mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala:230:24: "inital" is a misspelling of "initial" mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean" mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala:237:26: "descripiton" is a misspelling of "descriptions" python/pyspark/find_spark_home.py:30:13: "enviroment" is a misspelling of "environment" python/pyspark/context.py:937:12: "supress" is a misspelling of "suppress" python/pyspark/context.py:938:12: "supress" is a misspelling of "suppress" python/pyspark/context.py:939:12: "supress" is a misspelling of "suppress" python/pyspark/context.py:940:12: "supress" is a misspelling of "suppress" python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching" python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics" python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching" python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics" python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching" python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics" python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING" python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS" python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING" python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS" python/pyspark/heapq3.py:713:8: "probabilty" is a misspelling of "probability" python/pyspark/ml/clustering.py:1038:8: "Currenlty" is a misspelling of "Currently" python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean" python/pyspark/ml/regression.py:1378:20: "paramter" is a misspelling of "parameter" python/pyspark/mllib/stat/_statistics.py:262:8: "probabilty" is a misspelling of "probability" python/pyspark/rdd.py:1363:32: "paramter" is a misspelling of "parameter" python/pyspark/streaming/tests.py:825:42: "retuns" is a misspelling of "returns" python/pyspark/sql/tests.py:768:29: "initalization" is a misspelling of "initialization" python/pyspark/sql/tests.py:3616:31: "initalize" is a misspelling of "initialize" resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala:120:39: "arbitary" is a misspelling of "arbitrary" resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala:26:45: "sucessfully" is a misspelling of "successfully" resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala:358:27: "constaints" is a misspelling of "constraints" resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala:111:24: "senstive" is a misspelling of "sensitive" sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:1063:5: "overwirte" is a misspelling of "overwrite" sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala:1348:17: "compatability" is a misspelling of "compatibility" sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:77:36: "paramter" is a misspelling of "parameter" sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:1374:22: "precendence" is a misspelling of "precedence" sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:238:27: "unnecassary" is a misspelling of "unnecessary" sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala:212:17: "whn" is a misspelling of "when" sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala:147:60: "timestmap" is a misspelling of "timestamp" sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala:150:45: "precentage" is a misspelling of "percentage" sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala:135:29: "infered" is a misspelling of "inferred" sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922:1:52: "occurance" is a misspelling of "occurrence" sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182:1:52: "occurance" is a misspelling of "occurrence" sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e:1:63: "occurance" is a misspelling of "occurrence" sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478:1:63: "occurance" is a misspelling of "occurrence" sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:9:79: "occurence" is a misspelling of "occurrence" sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:13:110: "occurence" is a misspelling of "occurrence" sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q:46:105: "distint" is a misspelling of "distinct" sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q:29:3: "Currenly" is a misspelling of "Currently" sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q:72:15: "existant" is a misspelling of "existent" sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q:25:3: "substraction" is a misspelling of "subtraction" sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q:16:51: "funtion" is a misspelling of "function" sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q:15:30: "issueing" is a misspelling of "issuing" sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala:669:52: "wiht" is a misspelling of "with" sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java:474:9: "Refering" is a misspelling of "Referring" ``` ### after ``` $ misspell . | grep -v '.js' common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred" core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc" core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture" data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous" licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching" licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics" licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching" licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics" licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching" licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics" licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING" licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS" licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING" licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS" licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching" licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics" licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching" licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics" licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching" licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics" licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING" licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS" licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING" licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS" mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the" mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels" mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean" mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean" mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The" mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean" python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching" python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics" python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching" python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics" python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching" python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics" python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING" python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS" python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING" python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS" python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean" ``` Closes #22070 from seratch/fix-typo. Authored-by: Kazuhiro Sera <seratch@gmail.com> Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-11 22:23:36 -04:00
The target number of executors computed by the dynamicAllocation can still be overridden
by the <code>spark.dynamicAllocation.minExecutors</code> and
<code>spark.dynamicAllocation.maxExecutors</code> settings
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>2.4.0</td>
</tr>
<tr>
<td><code>spark.dynamicAllocation.schedulerBacklogTimeout</code></td>
<td>1s</td>
<td>
If dynamic allocation is enabled and there have been pending tasks backlogged for more than
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
this duration, new executors will be requested. For more detail, see this
<a href="job-scheduling.html#resource-allocation-policy">description</a>.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>1.2.0</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>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>1.2.0</td>
</tr>
<tr>
<td><code>spark.dynamicAllocation.shuffleTracking.enabled</code></td>
<td><code>false</code></td>
<td>
Experimental. Enables shuffle file tracking for executors, which allows dynamic allocation
without the need for an external shuffle service. This option will try to keep alive executors
that are storing shuffle data for active jobs.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.dynamicAllocation.shuffleTimeout</code></td>
<td><code>infinity</code></td>
<td>
When shuffle tracking is enabled, controls the timeout for executors that are holding shuffle
data. The default value means that Spark will rely on the shuffles being garbage collected to be
able to release executors. If for some reason garbage collection is not cleaning up shuffles
quickly enough, this option can be used to control when to time out executors even when they are
storing shuffle data.
</td>
[SPARK-31002][CORE][DOC][FOLLOWUP] Add version information to the configuration of Core ### What changes were proposed in this pull request? This PR follows up https://github.com/apache/spark/pull/27847. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.yarn.isPython | 1.5.0 | SPARK-5479 | 38112905bc3b33f2ae75274afba1c30e116f6e46#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.task.cpus | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.dynamicAllocation.enabled | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.testing | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.minExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.initialExecutors | 1.3.0 | SPARK-4585 | b2047b55c5fc85de6b63276d8ab9610d2496e08b#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.maxExecutors | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.executorAllocationRatio | 2.4.0 | SPARK-22683 | 55c4ca88a3b093ee197a8689631be8d1fac1f10f#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.cachedExecutorIdleTimeout | 1.4.0 | SPARK-7955 | 6faaf15ba311bc3a79aae40a6c9c4befabb6889f#diff-b096353602813e47074ace09a3890d56 |   spark.dynamicAllocation.executorIdleTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.shuffleTracking.enabled | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.shuffleTimeout | 3.0.0 | SPARK-27963 | 2ddeff97d7329942a98ef363991eeabc3fa71a76#diff-6bdad48cfc34314e89599655442ff210 |   spark.dynamicAllocation.schedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.dynamicAllocation.sustainedSchedulerBacklogTimeout | 1.2.0 | SPARK-3795 | 8d59b37b02eb36f37bcefafb952519d7dca744ad#diff-364713d7776956cb8b0a771e9b62f82d |   spark.locality.wait | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-391214d132a0fb4478f4f9c2313d8966 |   spark.shuffle.service.enabled | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED | 3.0.0 | SPARK-27677 | e9f3f62b2c0f521f3cc23fef381fc6754853ad4f#diff-6bdad48cfc34314e89599655442ff210 | spark.shuffle.service.fetch.rdd.enabled spark.shuffle.service.db.enabled | 3.0.0 | SPARK-26288 | 8b0aa59218c209d39cbba5959302d8668b885cf6#diff-6bdad48cfc34314e89599655442ff210 |   spark.shuffle.service.port | 1.2.0 | SPARK-3796 | f55218aeb1e9d638df6229b36a59a15ce5363482#diff-2b643ea78c1add0381754b1f47eec132 |   spark.kerberos.keytab | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.principal | 3.0.0 | SPARK-25372 | 51540c2fa677658be954c820bc18ba748e4c8583#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.relogin.period | 3.0.0 | SPARK-23781 | 68dde3481ea458b0b8deeec2f99233c2d4c1e056#diff-6bdad48cfc34314e89599655442ff210 | spark.kerberos.renewal.credentials | 3.0.0 | SPARK-26595 | 2a67dbfbd341af166b1c85904875f26a6dea5ba8#diff-6bdad48cfc34314e89599655442ff210 |   spark.kerberos.access.hadoopFileSystems | 3.0.0 | SPARK-26766 | d0443a74d185ec72b747fa39994fa9a40ce974cf#diff-6bdad48cfc34314e89599655442ff210 |   spark.executor.instances | 1.0.0 | SPARK-1126 | 1617816090e7b20124a512a43860a21232ebf511#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.yarn.dist.pyFiles | 2.2.1 | SPARK-21714 | d10c9dc3f631a26dbbbd8f5c601ca2001a5d7c80#diff-6bdad48cfc34314e89599655442ff210 |   spark.task.maxDirectResultSize | 2.0.0 | SPARK-13830 | 2ef4c5963bff3574fe17e669d703b25ddd064e5d#diff-5a0de266c82b95adb47d9bca714e1f1b |   spark.task.maxFailures | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-264da78fe625d594eae59d1adabc8ae9 |   spark.task.reaper.enabled | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.killTimeout | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.pollingInterval | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.task.reaper.threadDump | 2.0.3 | SPARK-18761 | 678d91c1d2283d9965a39656af9d383bad093ba8#diff-5a0de266c82b95adb47d9bca714e1f1b | spark.blacklist.enabled | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.task.maxTaskAttemptsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedTasksPerExecutor | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedTasksPerExecutor | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.application.maxFailedExecutorsPerNode | 2.2.0 | SPARK-8425 | 93cdb8a7d0f124b4db069fd8242207c82e263c52#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.stage.maxFailedExecutorsPerNode | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.timeout | 2.1.0 | SPARK-17675 | 9ce7d3e542e786c62f047c13f3001e178f76e06a#diff-6bdad48cfc34314e89599655442ff210 |   spark.blacklist.killBlacklistedExecutors | 2.2.0 | SPARK-16554 | 6287c94f08200d548df5cc0a401b73b84f9968c4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.executorTaskBlacklistTime | 1.0.0 | None | ab747d39ddc7c8a314ed2fb26548fc5652af0d74#diff-bad3987c83bd22d46416d3dd9d208e76 | spark.blacklist.application.fetchFailure.enabled | 2.3.0 | SPARK-13669 and SPARK-20898 | 9e50a1d37a4cf0c34e20a7c1a910ceaff41535a2#diff-6bdad48cfc34314e89599655442ff210 |   spark.files.fetchFailure.unRegisterOutputOnHost | 2.3.0 | SPARK-19753 | dccc0aa3cf957c8eceac598ac81ac82f03b52105#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.eventqueue.capacity | 2.3.0 | SPARK-20887 | 629f38e171409da614fd635bd8dd951b7fde17a4#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.metrics.maxListenerClassesTimed | 2.3.0 | SPARK-20863 | 2a23cdd078a7409d0bb92cf27718995766c41b1d#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent | 3.0.0 | SPARK-30812 | 68d7edf9497bea2f73707d32ab55dd8e53088e7c#diff-6bdad48cfc34314e89599655442ff210 |   spark.scheduler.listenerbus.logSlowEvent.threshold | 3.0.0 | SPARK-29001 | 0346afa8fc348aa1b3f5110df747a64e3b2da388#diff-6bdad48cfc34314e89599655442ff210 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27852 from beliefer/add-version-to-core-config-part-two. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-11 20:52:20 -04:00
<td>3.0.0</td>
</tr>
</table>
[SPARK-26632][CORE] Separate Thread Configurations of Driver and Executor ## What changes were proposed in this pull request? For the below three thread configuration items applied to both driver and executor, spark.rpc.io.serverThreads spark.rpc.io.clientThreads spark.rpc.netty.dispatcher.numThreads, we separate them to driver specifics and executor specifics. spark.driver.rpc.io.serverThreads < - > spark.executor.rpc.io.serverThreads spark.driver.rpc.io.clientThreads < - > spark.executor.rpc.io.clientThreads spark.driver.rpc.netty.dispatcher.numThreads < - > spark.executor.rpc.netty.dispatcher.numThreads Spark reads these specifics first and fall back to the common configurations. ## How was this patch tested? We ran the SimpleMap app without shuffle for benchmark purpose to test Spark's scalability in HPC with omini-path NIC which has higher bandwidth than normal ethernet NIC. Spark's base version is 2.4.0. Spark ran in the Standalone mode. Driver was in a standalone node. After the separation, the performance is improved a lot in 256 nodes and 512 nodes. see below test results of SimpleMapTask before and after the enhancement. You can view the tables in the [JIRA](https://issues.apache.org/jira/browse/SPARK-26632) too. ds: spark.driver.rpc.io.serverThreads dc: spark.driver.rpc.io.clientThreads dd: spark.driver.rpc.netty.dispatcher.numThreads ed: spark.executor.rpc.netty.dispatcher.numThreads time: Overall Time (s) old time: Overall Time without Separation (s) **Before:** nodes | ds | dc | dd | ed | time -- |-- | -- | -- | -- | -- 128 nodes | 8 | 8 | 8 | 8 | 108 256 nodes | 8 | 8 | 8 | 8 | 196 512 nodes | 8 | 8 | 8 | 8 | 377 **After:** nodes | ds | dc | dd | ed | time | improvement -- | -- | -- | -- | -- | -- | -- 128 nodes | 15 | 15 | 10 | 30 | 107 | 0.9% 256 nodes | 12 | 15 | 10 | 30 | 159 | 18.8% 512 nodes | 12 | 15 | 10 | 30 | 283 | 24.9% Closes #23560 from zjf2012/thread_conf_separation. Authored-by: jiafu.zhang@intel.com <jiafu.zhang@intel.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-10 13:42:43 -04:00
### Thread Configurations
Depending on jobs and cluster configurations, we can set number of threads in several places in Spark to utilize
available resources efficiently to get better performance. Prior to Spark 3.0, these thread configurations apply
to all roles of Spark, such as driver, executor, worker and master. From Spark 3.0, we can configure threads in
finer granularity starting from driver and executor. Take RPC module as example in below table. For other modules,
like shuffle, just replace "rpc" with "shuffle" in the property names except
<code>spark.{driver|executor}.rpc.netty.dispatcher.numThreads</code>, which is only for RPC module.
<table class="table">
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
[SPARK-26632][CORE] Separate Thread Configurations of Driver and Executor ## What changes were proposed in this pull request? For the below three thread configuration items applied to both driver and executor, spark.rpc.io.serverThreads spark.rpc.io.clientThreads spark.rpc.netty.dispatcher.numThreads, we separate them to driver specifics and executor specifics. spark.driver.rpc.io.serverThreads < - > spark.executor.rpc.io.serverThreads spark.driver.rpc.io.clientThreads < - > spark.executor.rpc.io.clientThreads spark.driver.rpc.netty.dispatcher.numThreads < - > spark.executor.rpc.netty.dispatcher.numThreads Spark reads these specifics first and fall back to the common configurations. ## How was this patch tested? We ran the SimpleMap app without shuffle for benchmark purpose to test Spark's scalability in HPC with omini-path NIC which has higher bandwidth than normal ethernet NIC. Spark's base version is 2.4.0. Spark ran in the Standalone mode. Driver was in a standalone node. After the separation, the performance is improved a lot in 256 nodes and 512 nodes. see below test results of SimpleMapTask before and after the enhancement. You can view the tables in the [JIRA](https://issues.apache.org/jira/browse/SPARK-26632) too. ds: spark.driver.rpc.io.serverThreads dc: spark.driver.rpc.io.clientThreads dd: spark.driver.rpc.netty.dispatcher.numThreads ed: spark.executor.rpc.netty.dispatcher.numThreads time: Overall Time (s) old time: Overall Time without Separation (s) **Before:** nodes | ds | dc | dd | ed | time -- |-- | -- | -- | -- | -- 128 nodes | 8 | 8 | 8 | 8 | 108 256 nodes | 8 | 8 | 8 | 8 | 196 512 nodes | 8 | 8 | 8 | 8 | 377 **After:** nodes | ds | dc | dd | ed | time | improvement -- | -- | -- | -- | -- | -- | -- 128 nodes | 15 | 15 | 10 | 30 | 107 | 0.9% 256 nodes | 12 | 15 | 10 | 30 | 159 | 18.8% 512 nodes | 12 | 15 | 10 | 30 | 283 | 24.9% Closes #23560 from zjf2012/thread_conf_separation. Authored-by: jiafu.zhang@intel.com <jiafu.zhang@intel.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-10 13:42:43 -04:00
<tr>
<td><code>spark.{driver|executor}.rpc.io.serverThreads</code></td>
<td>
Fall back on <code>spark.rpc.io.serverThreads</code>
</td>
<td>Number of threads used in the server thread pool</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.6.0</td>
[SPARK-26632][CORE] Separate Thread Configurations of Driver and Executor ## What changes were proposed in this pull request? For the below three thread configuration items applied to both driver and executor, spark.rpc.io.serverThreads spark.rpc.io.clientThreads spark.rpc.netty.dispatcher.numThreads, we separate them to driver specifics and executor specifics. spark.driver.rpc.io.serverThreads < - > spark.executor.rpc.io.serverThreads spark.driver.rpc.io.clientThreads < - > spark.executor.rpc.io.clientThreads spark.driver.rpc.netty.dispatcher.numThreads < - > spark.executor.rpc.netty.dispatcher.numThreads Spark reads these specifics first and fall back to the common configurations. ## How was this patch tested? We ran the SimpleMap app without shuffle for benchmark purpose to test Spark's scalability in HPC with omini-path NIC which has higher bandwidth than normal ethernet NIC. Spark's base version is 2.4.0. Spark ran in the Standalone mode. Driver was in a standalone node. After the separation, the performance is improved a lot in 256 nodes and 512 nodes. see below test results of SimpleMapTask before and after the enhancement. You can view the tables in the [JIRA](https://issues.apache.org/jira/browse/SPARK-26632) too. ds: spark.driver.rpc.io.serverThreads dc: spark.driver.rpc.io.clientThreads dd: spark.driver.rpc.netty.dispatcher.numThreads ed: spark.executor.rpc.netty.dispatcher.numThreads time: Overall Time (s) old time: Overall Time without Separation (s) **Before:** nodes | ds | dc | dd | ed | time -- |-- | -- | -- | -- | -- 128 nodes | 8 | 8 | 8 | 8 | 108 256 nodes | 8 | 8 | 8 | 8 | 196 512 nodes | 8 | 8 | 8 | 8 | 377 **After:** nodes | ds | dc | dd | ed | time | improvement -- | -- | -- | -- | -- | -- | -- 128 nodes | 15 | 15 | 10 | 30 | 107 | 0.9% 256 nodes | 12 | 15 | 10 | 30 | 159 | 18.8% 512 nodes | 12 | 15 | 10 | 30 | 283 | 24.9% Closes #23560 from zjf2012/thread_conf_separation. Authored-by: jiafu.zhang@intel.com <jiafu.zhang@intel.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-10 13:42:43 -04:00
</tr>
<tr>
<td><code>spark.{driver|executor}.rpc.io.clientThreads</code></td>
<td>
Fall back on <code>spark.rpc.io.clientThreads</code>
</td>
<td>Number of threads used in the client thread pool</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.6.0</td>
[SPARK-26632][CORE] Separate Thread Configurations of Driver and Executor ## What changes were proposed in this pull request? For the below three thread configuration items applied to both driver and executor, spark.rpc.io.serverThreads spark.rpc.io.clientThreads spark.rpc.netty.dispatcher.numThreads, we separate them to driver specifics and executor specifics. spark.driver.rpc.io.serverThreads < - > spark.executor.rpc.io.serverThreads spark.driver.rpc.io.clientThreads < - > spark.executor.rpc.io.clientThreads spark.driver.rpc.netty.dispatcher.numThreads < - > spark.executor.rpc.netty.dispatcher.numThreads Spark reads these specifics first and fall back to the common configurations. ## How was this patch tested? We ran the SimpleMap app without shuffle for benchmark purpose to test Spark's scalability in HPC with omini-path NIC which has higher bandwidth than normal ethernet NIC. Spark's base version is 2.4.0. Spark ran in the Standalone mode. Driver was in a standalone node. After the separation, the performance is improved a lot in 256 nodes and 512 nodes. see below test results of SimpleMapTask before and after the enhancement. You can view the tables in the [JIRA](https://issues.apache.org/jira/browse/SPARK-26632) too. ds: spark.driver.rpc.io.serverThreads dc: spark.driver.rpc.io.clientThreads dd: spark.driver.rpc.netty.dispatcher.numThreads ed: spark.executor.rpc.netty.dispatcher.numThreads time: Overall Time (s) old time: Overall Time without Separation (s) **Before:** nodes | ds | dc | dd | ed | time -- |-- | -- | -- | -- | -- 128 nodes | 8 | 8 | 8 | 8 | 108 256 nodes | 8 | 8 | 8 | 8 | 196 512 nodes | 8 | 8 | 8 | 8 | 377 **After:** nodes | ds | dc | dd | ed | time | improvement -- | -- | -- | -- | -- | -- | -- 128 nodes | 15 | 15 | 10 | 30 | 107 | 0.9% 256 nodes | 12 | 15 | 10 | 30 | 159 | 18.8% 512 nodes | 12 | 15 | 10 | 30 | 283 | 24.9% Closes #23560 from zjf2012/thread_conf_separation. Authored-by: jiafu.zhang@intel.com <jiafu.zhang@intel.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-10 13:42:43 -04:00
</tr>
<tr>
<td><code>spark.{driver|executor}.rpc.netty.dispatcher.numThreads</code></td>
<td>
Fall back on <code>spark.rpc.netty.dispatcher.numThreads</code>
</td>
<td>Number of threads used in RPC message dispatcher thread pool</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>3.0.0</td>
[SPARK-26632][CORE] Separate Thread Configurations of Driver and Executor ## What changes were proposed in this pull request? For the below three thread configuration items applied to both driver and executor, spark.rpc.io.serverThreads spark.rpc.io.clientThreads spark.rpc.netty.dispatcher.numThreads, we separate them to driver specifics and executor specifics. spark.driver.rpc.io.serverThreads < - > spark.executor.rpc.io.serverThreads spark.driver.rpc.io.clientThreads < - > spark.executor.rpc.io.clientThreads spark.driver.rpc.netty.dispatcher.numThreads < - > spark.executor.rpc.netty.dispatcher.numThreads Spark reads these specifics first and fall back to the common configurations. ## How was this patch tested? We ran the SimpleMap app without shuffle for benchmark purpose to test Spark's scalability in HPC with omini-path NIC which has higher bandwidth than normal ethernet NIC. Spark's base version is 2.4.0. Spark ran in the Standalone mode. Driver was in a standalone node. After the separation, the performance is improved a lot in 256 nodes and 512 nodes. see below test results of SimpleMapTask before and after the enhancement. You can view the tables in the [JIRA](https://issues.apache.org/jira/browse/SPARK-26632) too. ds: spark.driver.rpc.io.serverThreads dc: spark.driver.rpc.io.clientThreads dd: spark.driver.rpc.netty.dispatcher.numThreads ed: spark.executor.rpc.netty.dispatcher.numThreads time: Overall Time (s) old time: Overall Time without Separation (s) **Before:** nodes | ds | dc | dd | ed | time -- |-- | -- | -- | -- | -- 128 nodes | 8 | 8 | 8 | 8 | 108 256 nodes | 8 | 8 | 8 | 8 | 196 512 nodes | 8 | 8 | 8 | 8 | 377 **After:** nodes | ds | dc | dd | ed | time | improvement -- | -- | -- | -- | -- | -- | -- 128 nodes | 15 | 15 | 10 | 30 | 107 | 0.9% 256 nodes | 12 | 15 | 10 | 30 | 159 | 18.8% 512 nodes | 12 | 15 | 10 | 30 | 283 | 24.9% Closes #23560 from zjf2012/thread_conf_separation. Authored-by: jiafu.zhang@intel.com <jiafu.zhang@intel.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-10 13:42:43 -04:00
</tr>
</table>
The default value for number of thread-related config keys is the minimum of the number of cores requested for
the driver or executor, or, in the absence of that value, the number of cores available for the JVM (with a hardcoded upper limit of 8).
### Security
Please refer to the [Security](security.html) page for available options on how to secure different
Spark subsystems.
Spark 3883: SSL support for HttpServer and Akka SPARK-3883: SSL support for Akka connections and Jetty based file servers. This story introduced the following changes: - Introduced SSLOptions object which holds the SSL configuration and can build the appropriate configuration for Akka or Jetty. SSLOptions can be created by parsing SparkConf entries at a specified namespace. - SSLOptions is created and kept by SecurityManager - All Akka actor address creation snippets based on interpolated strings were replaced by a dedicated methods from AkkaUtils. Those methods select the proper Akka protocol - whether akka.tcp or akka.ssl.tcp - Added tests cases for AkkaUtils, FileServer, SSLOptions and SecurityManager - Added a way to use node local SSL configuration by executors and driver in standalone mode. It can be done by specifying spark.ssl.useNodeLocalConf in SparkConf. - Made CoarseGrainedExecutorBackend not overwrite the settings which are executor startup configuration - they are passed anyway from Worker Refer to https://github.com/apache/spark/pull/3571 for discussion and details Author: Jacek Lewandowski <lewandowski.jacek@gmail.com> Author: Jacek Lewandowski <jacek.lewandowski@datastax.com> Closes #3571 from jacek-lewandowski/SPARK-3883-master and squashes the following commits: 9ef4ed1 [Jacek Lewandowski] Merge pull request #2 from jacek-lewandowski/SPARK-3883-docs2 fb31b49 [Jacek Lewandowski] SPARK-3883: Added SSL setup documentation 2532668 [Jacek Lewandowski] SPARK-3883: Refactored AkkaUtils.protocol method to not use Try 90a8762 [Jacek Lewandowski] SPARK-3883: Refactored methods to resolve Akka address and made it possible to easily configure multiple communication layers for SSL 72b2541 [Jacek Lewandowski] SPARK-3883: A reference to the fallback SSLOptions can be provided when constructing SSLOptions 93050f4 [Jacek Lewandowski] SPARK-3883: SSL support for HttpServer and Akka
2015-02-02 20:18:54 -05:00
{% for static_file in site.static_files %}
{% if static_file.name == 'sql-configs.html' %}
### Spark SQL
{% include_relative sql-configs.html %}
{% break %}
{% endif %}
{% endfor %}
### Spark Streaming
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<table class="table">
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<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).
This enables the Spark Streaming to control the receiving rate based on the
current batch scheduling delays and processing times so that the system receives
only as fast as the system can process. Internally, this dynamically sets the
maximum receiving rate of receivers. This rate is upper bounded by the values
<code>spark.streaming.receiver.maxRate</code> and <code>spark.streaming.kafka.maxRatePerPartition</code>
if they are set (see below).
</td>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td>1.5.0</td>
</tr>
<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>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td>2.0.0</td>
</tr>
Clean up and simplify Spark configuration Over time as we've added more deployment modes, this have gotten a bit unwieldy with user-facing configuration options in Spark. Going forward we'll advise all users to run `spark-submit` to launch applications. This is a WIP patch but it makes the following improvements: 1. Improved `spark-env.sh.template` which was missing a lot of things users now set in that file. 2. Removes the shipping of SPARK_CLASSPATH, SPARK_JAVA_OPTS, and SPARK_LIBRARY_PATH to the executors on the cluster. This was an ugly hack. Instead it introduces config variables spark.executor.extraJavaOpts, spark.executor.extraLibraryPath, and spark.executor.extraClassPath. 3. Adds ability to set these same variables for the driver using `spark-submit`. 4. Allows you to load system properties from a `spark-defaults.conf` file when running `spark-submit`. This will allow setting both SparkConf options and other system properties utilized by `spark-submit`. 5. Made `SPARK_LOCAL_IP` an environment variable rather than a SparkConf property. This is more consistent with it being set on each node. Author: Patrick Wendell <pwendell@gmail.com> Closes #299 from pwendell/config-cleanup and squashes the following commits: 127f301 [Patrick Wendell] Improvements to testing a006464 [Patrick Wendell] Moving properties file template. b4b496c [Patrick Wendell] spark-defaults.properties -> spark-defaults.conf 0086939 [Patrick Wendell] Minor style fixes af09e3e [Patrick Wendell] Mention config file in docs and clean-up docs b16e6a2 [Patrick Wendell] Cleanup of spark-submit script and Scala quick start guide af0adf7 [Patrick Wendell] Automatically add user jar a56b125 [Patrick Wendell] Responses to Tom's review d50c388 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a762901 [Patrick Wendell] Fixing test failures ffa00fe [Patrick Wendell] Review feedback fda0301 [Patrick Wendell] Note 308f1f6 [Patrick Wendell] Properly escape quotes and other clean-up for YARN e83cd8f [Patrick Wendell] Changes to allow re-use of test applications be42f35 [Patrick Wendell] Handle case where SPARK_HOME is not set c2a2909 [Patrick Wendell] Test compile fixes 4ee6f9d [Patrick Wendell] Making YARN doc changes consistent afc9ed8 [Patrick Wendell] Cleaning up line limits and two compile errors. b08893b [Patrick Wendell] Additional improvements. ace4ead [Patrick Wendell] Responses to review feedback. b72d183 [Patrick Wendell] Review feedback for spark env file 46555c1 [Patrick Wendell] Review feedback and import clean-ups 437aed1 [Patrick Wendell] Small fix 761ebcd [Patrick Wendell] Library path and classpath for drivers 7cc70e4 [Patrick Wendell] Clean up terminology inside of spark-env script 5b0ba8e [Patrick Wendell] Don't ship executor envs 84cc5e5 [Patrick Wendell] Small clean-up 1f75238 [Patrick Wendell] SPARK_JAVA_OPTS --> SPARK_MASTER_OPTS for master settings 4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH 6eaf7d0 [Patrick Wendell] executorJavaOpts 0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS
2014-04-21 13:26:33 -04:00
<tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
<td><code>spark.streaming.blockInterval</code></td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
<td>200ms</td>
Clean up and simplify Spark configuration Over time as we've added more deployment modes, this have gotten a bit unwieldy with user-facing configuration options in Spark. Going forward we'll advise all users to run `spark-submit` to launch applications. This is a WIP patch but it makes the following improvements: 1. Improved `spark-env.sh.template` which was missing a lot of things users now set in that file. 2. Removes the shipping of SPARK_CLASSPATH, SPARK_JAVA_OPTS, and SPARK_LIBRARY_PATH to the executors on the cluster. This was an ugly hack. Instead it introduces config variables spark.executor.extraJavaOpts, spark.executor.extraLibraryPath, and spark.executor.extraClassPath. 3. Adds ability to set these same variables for the driver using `spark-submit`. 4. Allows you to load system properties from a `spark-defaults.conf` file when running `spark-submit`. This will allow setting both SparkConf options and other system properties utilized by `spark-submit`. 5. Made `SPARK_LOCAL_IP` an environment variable rather than a SparkConf property. This is more consistent with it being set on each node. Author: Patrick Wendell <pwendell@gmail.com> Closes #299 from pwendell/config-cleanup and squashes the following commits: 127f301 [Patrick Wendell] Improvements to testing a006464 [Patrick Wendell] Moving properties file template. b4b496c [Patrick Wendell] spark-defaults.properties -> spark-defaults.conf 0086939 [Patrick Wendell] Minor style fixes af09e3e [Patrick Wendell] Mention config file in docs and clean-up docs b16e6a2 [Patrick Wendell] Cleanup of spark-submit script and Scala quick start guide af0adf7 [Patrick Wendell] Automatically add user jar a56b125 [Patrick Wendell] Responses to Tom's review d50c388 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a762901 [Patrick Wendell] Fixing test failures ffa00fe [Patrick Wendell] Review feedback fda0301 [Patrick Wendell] Note 308f1f6 [Patrick Wendell] Properly escape quotes and other clean-up for YARN e83cd8f [Patrick Wendell] Changes to allow re-use of test applications be42f35 [Patrick Wendell] Handle case where SPARK_HOME is not set c2a2909 [Patrick Wendell] Test compile fixes 4ee6f9d [Patrick Wendell] Making YARN doc changes consistent afc9ed8 [Patrick Wendell] Cleaning up line limits and two compile errors. b08893b [Patrick Wendell] Additional improvements. ace4ead [Patrick Wendell] Responses to review feedback. b72d183 [Patrick Wendell] Review feedback for spark env file 46555c1 [Patrick Wendell] Review feedback and import clean-ups 437aed1 [Patrick Wendell] Small fix 761ebcd [Patrick Wendell] Library path and classpath for drivers 7cc70e4 [Patrick Wendell] Clean up terminology inside of spark-env script 5b0ba8e [Patrick Wendell] Don't ship executor envs 84cc5e5 [Patrick Wendell] Small clean-up 1f75238 [Patrick Wendell] SPARK_JAVA_OPTS --> SPARK_MASTER_OPTS for master settings 4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH 6eaf7d0 [Patrick Wendell] executorJavaOpts 0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS
2014-04-21 13:26:33 -04:00
<td>
[SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Author: Ilya Ganelin <ilganeli@gmail.com> Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time
2015-04-13 19:28:07 -04:00
Interval at which data received by Spark Streaming receivers is chunked
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
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 programming guide for more details.
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</td>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td>0.8.0</td>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
</tr>
<tr>
<td><code>spark.streaming.receiver.maxRate</code></td>
<td>not set</td>
<td>
Maximum rate (number of records per second) at which each receiver will receive data.
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
Effectively, each stream will consume at most this number of records per second.
Setting this configuration to 0 or a negative number will put no limit on the rate.
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
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 programming guide for mode details.
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
</td>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td>1.0.2</td>
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
2014-12-11 09:21:23 -05:00
</tr>
<tr>
<td><code>spark.streaming.receiver.writeAheadLog.enable</code></td>
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
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.
[SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Author: Josh Rosen <rosenville@gmail.com> Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section.
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 programming guide for more details.
</td>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td>1.2.1</td>
</tr>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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.
Clean up and simplify Spark configuration Over time as we've added more deployment modes, this have gotten a bit unwieldy with user-facing configuration options in Spark. Going forward we'll advise all users to run `spark-submit` to launch applications. This is a WIP patch but it makes the following improvements: 1. Improved `spark-env.sh.template` which was missing a lot of things users now set in that file. 2. Removes the shipping of SPARK_CLASSPATH, SPARK_JAVA_OPTS, and SPARK_LIBRARY_PATH to the executors on the cluster. This was an ugly hack. Instead it introduces config variables spark.executor.extraJavaOpts, spark.executor.extraLibraryPath, and spark.executor.extraClassPath. 3. Adds ability to set these same variables for the driver using `spark-submit`. 4. Allows you to load system properties from a `spark-defaults.conf` file when running `spark-submit`. This will allow setting both SparkConf options and other system properties utilized by `spark-submit`. 5. Made `SPARK_LOCAL_IP` an environment variable rather than a SparkConf property. This is more consistent with it being set on each node. Author: Patrick Wendell <pwendell@gmail.com> Closes #299 from pwendell/config-cleanup and squashes the following commits: 127f301 [Patrick Wendell] Improvements to testing a006464 [Patrick Wendell] Moving properties file template. b4b496c [Patrick Wendell] spark-defaults.properties -> spark-defaults.conf 0086939 [Patrick Wendell] Minor style fixes af09e3e [Patrick Wendell] Mention config file in docs and clean-up docs b16e6a2 [Patrick Wendell] Cleanup of spark-submit script and Scala quick start guide af0adf7 [Patrick Wendell] Automatically add user jar a56b125 [Patrick Wendell] Responses to Tom's review d50c388 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a762901 [Patrick Wendell] Fixing test failures ffa00fe [Patrick Wendell] Review feedback fda0301 [Patrick Wendell] Note 308f1f6 [Patrick Wendell] Properly escape quotes and other clean-up for YARN e83cd8f [Patrick Wendell] Changes to allow re-use of test applications be42f35 [Patrick Wendell] Handle case where SPARK_HOME is not set c2a2909 [Patrick Wendell] Test compile fixes 4ee6f9d [Patrick Wendell] Making YARN doc changes consistent afc9ed8 [Patrick Wendell] Cleaning up line limits and two compile errors. b08893b [Patrick Wendell] Additional improvements. ace4ead [Patrick Wendell] Responses to review feedback. b72d183 [Patrick Wendell] Review feedback for spark env file 46555c1 [Patrick Wendell] Review feedback and import clean-ups 437aed1 [Patrick Wendell] Small fix 761ebcd [Patrick Wendell] Library path and classpath for drivers 7cc70e4 [Patrick Wendell] Clean up terminology inside of spark-env script 5b0ba8e [Patrick Wendell] Don't ship executor envs 84cc5e5 [Patrick Wendell] Small clean-up 1f75238 [Patrick Wendell] SPARK_JAVA_OPTS --> SPARK_MASTER_OPTS for master settings 4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH 6eaf7d0 [Patrick Wendell] executorJavaOpts 0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS
2014-04-21 13:26:33 -04:00
</td>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td>0.9.0</td>
Clean up and simplify Spark configuration Over time as we've added more deployment modes, this have gotten a bit unwieldy with user-facing configuration options in Spark. Going forward we'll advise all users to run `spark-submit` to launch applications. This is a WIP patch but it makes the following improvements: 1. Improved `spark-env.sh.template` which was missing a lot of things users now set in that file. 2. Removes the shipping of SPARK_CLASSPATH, SPARK_JAVA_OPTS, and SPARK_LIBRARY_PATH to the executors on the cluster. This was an ugly hack. Instead it introduces config variables spark.executor.extraJavaOpts, spark.executor.extraLibraryPath, and spark.executor.extraClassPath. 3. Adds ability to set these same variables for the driver using `spark-submit`. 4. Allows you to load system properties from a `spark-defaults.conf` file when running `spark-submit`. This will allow setting both SparkConf options and other system properties utilized by `spark-submit`. 5. Made `SPARK_LOCAL_IP` an environment variable rather than a SparkConf property. This is more consistent with it being set on each node. Author: Patrick Wendell <pwendell@gmail.com> Closes #299 from pwendell/config-cleanup and squashes the following commits: 127f301 [Patrick Wendell] Improvements to testing a006464 [Patrick Wendell] Moving properties file template. b4b496c [Patrick Wendell] spark-defaults.properties -> spark-defaults.conf 0086939 [Patrick Wendell] Minor style fixes af09e3e [Patrick Wendell] Mention config file in docs and clean-up docs b16e6a2 [Patrick Wendell] Cleanup of spark-submit script and Scala quick start guide af0adf7 [Patrick Wendell] Automatically add user jar a56b125 [Patrick Wendell] Responses to Tom's review d50c388 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a762901 [Patrick Wendell] Fixing test failures ffa00fe [Patrick Wendell] Review feedback fda0301 [Patrick Wendell] Note 308f1f6 [Patrick Wendell] Properly escape quotes and other clean-up for YARN e83cd8f [Patrick Wendell] Changes to allow re-use of test applications be42f35 [Patrick Wendell] Handle case where SPARK_HOME is not set c2a2909 [Patrick Wendell] Test compile fixes 4ee6f9d [Patrick Wendell] Making YARN doc changes consistent afc9ed8 [Patrick Wendell] Cleaning up line limits and two compile errors. b08893b [Patrick Wendell] Additional improvements. ace4ead [Patrick Wendell] Responses to review feedback. b72d183 [Patrick Wendell] Review feedback for spark env file 46555c1 [Patrick Wendell] Review feedback and import clean-ups 437aed1 [Patrick Wendell] Small fix 761ebcd [Patrick Wendell] Library path and classpath for drivers 7cc70e4 [Patrick Wendell] Clean up terminology inside of spark-env script 5b0ba8e [Patrick Wendell] Don't ship executor envs 84cc5e5 [Patrick Wendell] Small clean-up 1f75238 [Patrick Wendell] SPARK_JAVA_OPTS --> SPARK_MASTER_OPTS for master settings 4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH 6eaf7d0 [Patrick Wendell] executorJavaOpts 0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS
2014-04-21 13:26:33 -04:00
</tr>
<tr>
<td><code>spark.streaming.stopGracefullyOnShutdown</code></td>
<td>false</td>
<td>
If <code>true</code>, Spark shuts down the <code>StreamingContext</code> gracefully on JVM
shutdown rather than immediately.
</td>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td>1.4.0</td>
</tr>
<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-0-10-integration.html">Kafka Integration guide</a>
for more details.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>1.3.0</td>
</tr>
<tr>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td><code>spark.streaming.kafka.minRatePerPartition</code></td>
<td>1</td>
<td>
Minimum rate (number of records per second) at which data will be read from each Kafka
partition when using the new Kafka direct stream API.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>2.4.0</td>
</tr>
<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>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td>1.0.0</td>
</tr>
<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>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td>1.6.0</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>
[SPARK-31141][DSTREAMS][DOC] Add version information to the configuration of Dstreams ### What changes were proposed in this pull request? Add version information to the configuration of `Dstreams`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.backpressure.enabled | 1.5.0 | SPARK-9967 and SPARK-10099 | 392bd19d678567751cd3844d9d166a7491c5887e#diff-1b584c4ed88a9022abb11d594f760997 |   spark.streaming.backpressure.initialRate | 2.0.0 | SPARK-11627 | 7218c0eba957e0a079a407b79c3a050cce9647b2#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.blockInterval | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-54d85b29e4349628a0de525c119399b5 |   spark.streaming.receiver.maxRate | 1.0.2 | SPARK-1341 | ca19cfbcd5cfac9ad731350dfeea14355aec87d6#diff-c64d571ef32d2dbf76e965ecd04a9f52 |   spark.streaming.receiver.writeAheadLog.enable | 1.2.1 | SPARK-4482 | ce5ea0fd611ce560f6e1fac83562469bdb97091e#diff-0607b70e4e79cbbc1a128c45784cb813 |   spark.streaming.unpersist | 0.9.0 | None | 08b9fec93d00ff0ebb49af4d9ac72d2806eded02#diff-bcf5f84f78d23ebde7d532bea756bc57 |   spark.streaming.stopGracefullyOnShutdown | 1.4.0 | SPARK-7776 | a17a5cb302c5fa6a4d3e9e3e0fa2100c0b5436d6#diff-8a7f0e3f26c15ba484e6312c3caf033d |   spark.streaming.kafka.maxRetries | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.streaming.driver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.closeFileAfterWrite | 1.6.0 | SPARK-11324 | 4f030b9e82172659d250281782ac573cbd1438fc#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.receiver.writeAheadLog.maxFailures | 1.2.0 | SPARK-4028 | 234de9232bcfa212317a8073c4a82c3863b36b14#diff-8cec1a581eebcad673dc8930b1a2801c |   spark.streaming.driver.writeAheadLog.class | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.rollingIntervalSecs | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.maxFailures | 1.4.0 | SPARK-7056 | 1868bd40dcce23990b98748b0239bd00452b1ca5#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.allowBatching | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.driver.writeAheadLog.batchingTimeout | 1.6.0 | SPARK-11141 | dccc4645df629f35c4788d50b2c0a6ab381db4b7#diff-a1b3ec72e8d7cc91433a1cc64fe6e91d |   spark.streaming.sessionByKey.deltaChainThreshold | 1.6.0 | SPARK-11290 | daa74be6f863061221bb0c2f94e70672e6fcbeaa#diff-e0a40541298f885606a2361ff9c5af6c |   spark.streaming.backpressure.rateEstimator | 1.5.0 | SPARK-8977 | 819be46e5a73f2d19230354ebba30c58538590f5#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.proportional | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.integral | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.derived | 1.5.0 | SPARK-8979 | 0a1d2ca42c8b31d6b0e70163795f0185d4622f87#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.backpressure.pid.minRate | 1.5.0 | SPARK-9966 | 612b4609bdd38763725ae07d77c2176aa6756e64#diff-5dcaea3a4eca07f898fa88fe6d69e5c3 |   spark.streaming.concurrentJobs | 0.7.0 | None | c97ebf64377e853ab7c616a103869a4417f25954#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.internal.batchTime | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.internal.outputOpId | 1.4.0 | SPARK-6862 | 1b7106b867bc0aa4d64b669d79b646f862acaf47#diff-25124e4f06a1da237bf486eceb1f7967 | It's not a configuration, it's a property spark.streaming.clock | 0.7.0 | None | cae894ee7aefa4cf9b1952038a48be81e1d2a856#diff-839f06302b2d648a85436486fc13c85d |   spark.streaming.gracefulStopTimeout | 1.0.0 | SPARK-1332 | 94cbe2329021296b660d88f3e8ef3734374020d2#diff-2f8c5c038fda47b9875e10785fdd2498 |   spark.streaming.manualClock.jump | 0.7.0 | None | fc3d0b602a08fdd182c2138506d1cd9952631f95#diff-839f06302b2d648a85436486fc13c85d |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists UT Closes #27898 from beliefer/add-version-to-dstream-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-23 00:01:44 -04:00
<td>1.6.0</td>
</tr>
</table>
### SparkR
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</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>
<td>1.4.0</td>
</tr>
<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>
<td>1.5.3</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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>1.5.3</td>
</tr>
<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>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>2.1.0</td>
</tr>
<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>
<td>2.1.0</td>
</tr>
<tr>
<td><code>spark.r.heartBeatInterval</code></td>
<td>100</td>
<td>
Interval for heartbeats sent from SparkR backend to R process to prevent connection timeout.
</td>
<td>2.1.0</td>
</tr>
</table>
### GraphX
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
<tr>
<td><code>spark.graphx.pregel.checkpointInterval</code></td>
<td>-1</td>
<td>
Checkpoint interval for graph and message in Pregel. It used to avoid stackOverflowError due to long lineage chains
after lots of iterations. The checkpoint is disabled by default.
</td>
[SPARK-31269][DOC] Supplement version for configuration only appear in configuration doc ### What changes were proposed in this pull request? The `configuration.md` exists some config not organized by `ConfigEntry`. This PR supplements version for configuration only appear in configuration doc. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.app.name | 0.9.0 | None | 994f080f8ae3372366e6004600ba791c8a372ff0#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.discoveryScript | 3.0.0 | SPARK-27488 | 74e5e41eebf9ed596b48e6db52a2a9c642e5cbc3#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.driver.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceType}.discoveryScript | 3.0.0 | SPARK-27024 | db2e3c43412e4a7fb4a46c58d73d9ab304a1e949#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.executor.resource.{resourceName}.vendor | 3.0.0 | SPARK-27362 | 1277f8fa92da85d9e39d9146e3099fcb75c71a8f#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.local.dir | 0.5.0 | None | 0e93891d3d7df849cff6442038c111ffd42a5243#diff-17fd275d280b667722664ed833c6402a |   spark.logConf | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-364713d7776956cb8b0a771e9b62f82d |   spark.master | 0.9.0 | SPARK-544 | 2573add94cf920a88f74d80d8ea94218d812704d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.driver.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executor.defaultJavaOptions | 3.0.0 | SPARK-23472 | f83000597f250868de9722d8285fed013abc5ecf#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.executorEnv.[EnvironmentVariableName] | 0.9.0 | None | 642029e7f43322f84abe4f7f36bb0b1b95d8101d#diff-529fc5c06b9731c1fbda6f3db60b16aa |   spark.python.profile | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.profile.dump | 1.2.0 | SPARK-3478 | 1aa549ba9839565274a12c52fa1075b424f138a6#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.python.worker.memory | 1.1.0 | SPARK-2538 | 14174abd421318e71c16edd24224fd5094bdfed4#diff-d6fe2792e44f6babc94aabfefc8b9bce |   spark.jars.packages | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.excludes | 1.5.0 | SPARK-9263 | 34335719a372c1951fdb4dd25b75b086faf1076f#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivy | 1.3.0 | SPARK-5341 | 3b7acd22ab4a134c74746e3b9a803dbd34d43855#diff-63a5d817d2d45ae24de577f6a1bd80f9 |   spark.jars.ivySettings | 2.2.0 | SPARK-17568 | 3bc2eff8880a3ba8d4318118715ea1a47048e3de#diff-4d2ab44195558d5a9d5f15b8803ef39d |   spark.jars.repositories | 2.3.0 | SPARK-21403 | d8257b99ddae23f702f312640a5335ddb4554403#diff-4d2ab44195558d5a9d5f15b8803ef39d | spark.shuffle.io.maxRetries | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.numConnectionsPerPeer | 1.2.1 | SPARK-4740 | 441ec3451730c7ae3dbef8952e313071d6147ab6#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.preferDirectBufs | 1.2.0 | SPARK-4188 | c1ea5c542f3267c0b23a7775887e3a6ece793fe3#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.retryWait | 1.2.1 | None | 5e5d8f469a1bea9bbe606f772ccdcab7c184c651#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.shuffle.io.backLog | 1.1.1 | SPARK-2468 | 66b4c81db7e826c00f7fb449b8a8af810cf7dd9a#diff-bdee8e601924d41e93baa7287189e878 |   spark.shuffle.service.index.cache.size | 2.3.0 | SPARK-21501 | 1662e93119d68498942386906de309d35f4a135f#diff-97d5edc927a83a678e013ae00343df94 | spark.shuffle.maxChunksBeingTransferred | 2.3.0 | SPARK-21175 | 799e13161e89f1ea96cb1bc7b507a05af2e89cd0#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.sql.ui.retainedExecutions | 1.5.0 | SPARK-8861 and SPARK-8862 | ebc3aad272b91cf58e2e1b4aa92b49b8a947a045#diff-81764e4d52817f83bdd5336ef1226bd9 |   spark.streaming.ui.retainedBatches | 1.0.0 | SPARK-1386 | f36dc3fed0a0671b0712d664db859da28c0a98e2#diff-56b8d67d07284cfab165d5363bd3500e | spark.default.parallelism | 0.5.0 | None | e5c4cd8a5e188592f8786a265c0cd073c69ac886#diff-0544ebf7533fa70ff5103e0fe1f0b036 |   spark.files.fetchTimeout | 1.0.0 | None | f6f9d02e85d17da2f742ed0062f1648a9293e73c#diff-d239aee594001f8391676e1047a0381e |   spark.files.useFetchCache | 1.2.2 | SPARK-6313 | a2a94a154bdd00753b8d5e344d712664c7151050#diff-d239aee594001f8391676e1047a0381e | spark.files.overwrite | 1.0.0 | None | 84670f2715392859624df290c1b52eb4ed4a9cb1#diff-d239aee594001f8391676e1047a0381e | Exists in branch-1.0, but the version of pom is 0.9.0-incubating-SNAPSHOT spark.hadoop.cloneConf | 1.0.3 | SPARK-2546 | 6d8f1dd15afdc7432b5721c89f9b2b402460322b#diff-83eb37f7b0ebed3c14ccb7bff0d577c2 |   spark.hadoop.validateOutputSpecs | 1.0.1 | SPARK-1677 | 8100cbdb7546e8438019443cfc00683017c81278#diff-f70e97c099b5eac05c75288cb215e080 | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version | 2.2.0 | SPARK-20107 | edc87d76efea7b4d19d9d0c4ddba274a3ccb8752#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.rpc.io.backLog | 3.0.0 | SPARK-27868 | 09ed64d795d3199a94e175273fff6fcea6b52131#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.network.io.preferDirectBufs | 3.0.0 | SPARK-24920 | e103c4a5e72bab8862ff49d6d4c1e62e642fc412#diff-0ac65da2bc6b083fb861fe410c7688c2 |   spark.port.maxRetries | 1.1.1 | SPARK-3565 | 32f2222e915f31422089139944a077e2cbd442f9#diff-d239aee594001f8391676e1047a0381e |   spark.core.connection.ack.wait.timeout | 1.1.1 | SPARK-2677 | bd3ce2ffb8964abb4d59918ebb2c230fe4614aa2#diff-f748e95f2aa97ed715afa53ddeeac9de |   spark.scheduler.listenerbus.eventqueue.shared.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.appStatus.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.executorManagement.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.eventLog.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.scheduler.listenerbus.eventqueue.streams.capacity | 3.0.0 | SPARK-28574 | c212c9d9ed7375cd1ea16c118733edd84037ec0d#diff-eb519ad78cc3cf0b95839cc37413b509 |   spark.task.resource.{resourceName}.amount | 3.0.0 | SPARK-27760 | d30284b5a51dd784f663eb4eea37087b35a54d00#diff-76e731333fb756df3bff5ddb3b731c46 |   spark.stage.maxConsecutiveAttempts | 2.2.0 | SPARK-13369 | 7b5d873aef672aa0aee41e338bab7428101e1ad3#diff-6a9ff7fb74fd490a50462d45db2d5e11 |   spark.{driver\|executor}.rpc.io.serverThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.io.clientThreads | 1.6.0 | SPARK-10745 | 7c5b641808740ba5eed05ba8204cdbaf3fc579f5#diff-d2ce9b38bdc38ca9d7119f9c2cf79907 |   spark.{driver\|executor}.rpc.netty.dispatcher.numThreads | 3.0.0 | SPARK-29398 | 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931#diff-a68a21481fea5053848ca666dd3201d8 |   spark.r.driver.command | 1.5.3 | SPARK-10971 | 9695f452e86a88bef3bcbd1f3c0b00ad9e9ac6e1#diff-025470e1b7094d7cf4a78ea353fb3981 |   spark.r.shell.command | 2.1.0 | SPARK-17178 | fa6347938fc1c72ddc03a5f3cd2e929b5694f0a6#diff-a78ecfc6a89edfaf0b60a5eaa0381970 |   spark.graphx.pregel.checkpointInterval | 2.2.0 | SPARK-5484 | f971ce5dd0788fe7f5d2ca820b9ea3db72033ddc#diff-e399679417ffa6eeedf26a7630baca16 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28035 from beliefer/supplement-configuration-version. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-30 23:32:04 -04:00
<td>2.2.0</td>
</tr>
</table>
### Deploy
<table class="table">
[SPARK-30887][CORE][DOC] Add version information to the configuration of Deploy ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Deploy`. 2.Update the docs of `Deploy`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.deploy.recoveryMode | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.recoveryMode.factory | 1.2.0 | SPARK-1830 | deefd9d7377a8091a1d184b99066febd0e9f6afd#diff-29dffdccd5a7f4c8b496c293e87c8668 | This configuration appears in branch-1.3, but the version number in the pom.xml file corresponding to the commit is 1.2.0-SNAPSHOT spark.deploy.recoveryDirectory | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.zookeeper.url | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-4457313ca662a1cd60197122d924585c | spark.deploy.zookeeper.dir | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-a84228cb45c7d5bd93305a1f5bf720b6 | spark.deploy.retainedApplications | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.retainedDrivers | 1.1.0 | None | 7446f5ff93142d2dd5c79c63fa947f47a1d4db8b#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.dead.worker.persistence | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.maxExecutorRetries | 1.6.3 | SPARK-16956 | ace458f0330f22463ecf7cbee7c0465e10fba8a8#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.spreadOut | 0.6.1 | None | bb2b9ff37cd2503cc6ea82c5dd395187b0910af0#diff-0e7ae91819fc8f7b47b0f97be7116325 | spark.deploy.defaultCores | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-29dffdccd5a7f4c8b496c293e87c8668 | ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27668 from beliefer/add-version-to-deploy-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-24 21:39:11 -05:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<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>
[SPARK-30887][CORE][DOC] Add version information to the configuration of Deploy ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Deploy`. 2.Update the docs of `Deploy`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.deploy.recoveryMode | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.recoveryMode.factory | 1.2.0 | SPARK-1830 | deefd9d7377a8091a1d184b99066febd0e9f6afd#diff-29dffdccd5a7f4c8b496c293e87c8668 | This configuration appears in branch-1.3, but the version number in the pom.xml file corresponding to the commit is 1.2.0-SNAPSHOT spark.deploy.recoveryDirectory | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.zookeeper.url | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-4457313ca662a1cd60197122d924585c | spark.deploy.zookeeper.dir | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-a84228cb45c7d5bd93305a1f5bf720b6 | spark.deploy.retainedApplications | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.retainedDrivers | 1.1.0 | None | 7446f5ff93142d2dd5c79c63fa947f47a1d4db8b#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.dead.worker.persistence | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.maxExecutorRetries | 1.6.3 | SPARK-16956 | ace458f0330f22463ecf7cbee7c0465e10fba8a8#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.spreadOut | 0.6.1 | None | bb2b9ff37cd2503cc6ea82c5dd395187b0910af0#diff-0e7ae91819fc8f7b47b0f97be7116325 | spark.deploy.defaultCores | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-29dffdccd5a7f4c8b496c293e87c8668 | ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27668 from beliefer/add-version-to-deploy-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-24 21:39:11 -05:00
<td>0.8.1</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>
[SPARK-30887][CORE][DOC] Add version information to the configuration of Deploy ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Deploy`. 2.Update the docs of `Deploy`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.deploy.recoveryMode | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.recoveryMode.factory | 1.2.0 | SPARK-1830 | deefd9d7377a8091a1d184b99066febd0e9f6afd#diff-29dffdccd5a7f4c8b496c293e87c8668 | This configuration appears in branch-1.3, but the version number in the pom.xml file corresponding to the commit is 1.2.0-SNAPSHOT spark.deploy.recoveryDirectory | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.zookeeper.url | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-4457313ca662a1cd60197122d924585c | spark.deploy.zookeeper.dir | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-a84228cb45c7d5bd93305a1f5bf720b6 | spark.deploy.retainedApplications | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.retainedDrivers | 1.1.0 | None | 7446f5ff93142d2dd5c79c63fa947f47a1d4db8b#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.dead.worker.persistence | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.maxExecutorRetries | 1.6.3 | SPARK-16956 | ace458f0330f22463ecf7cbee7c0465e10fba8a8#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.spreadOut | 0.6.1 | None | bb2b9ff37cd2503cc6ea82c5dd395187b0910af0#diff-0e7ae91819fc8f7b47b0f97be7116325 | spark.deploy.defaultCores | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-29dffdccd5a7f4c8b496c293e87c8668 | ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27668 from beliefer/add-version-to-deploy-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-24 21:39:11 -05:00
<td>0.8.1</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>
[SPARK-30887][CORE][DOC] Add version information to the configuration of Deploy ### What changes were proposed in this pull request? 1.Add version information to the configuration of `Deploy`. 2.Update the docs of `Deploy`. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.deploy.recoveryMode | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.recoveryMode.factory | 1.2.0 | SPARK-1830 | deefd9d7377a8091a1d184b99066febd0e9f6afd#diff-29dffdccd5a7f4c8b496c293e87c8668 | This configuration appears in branch-1.3, but the version number in the pom.xml file corresponding to the commit is 1.2.0-SNAPSHOT spark.deploy.recoveryDirectory | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.zookeeper.url | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-4457313ca662a1cd60197122d924585c | spark.deploy.zookeeper.dir | 0.8.1 | None | d66c01f2b6defb3db6c1be99523b734a4d960532#diff-a84228cb45c7d5bd93305a1f5bf720b6 | spark.deploy.retainedApplications | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.retainedDrivers | 1.1.0 | None | 7446f5ff93142d2dd5c79c63fa947f47a1d4db8b#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.dead.worker.persistence | 0.8.0 | None | 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.maxExecutorRetries | 1.6.3 | SPARK-16956 | ace458f0330f22463ecf7cbee7c0465e10fba8a8#diff-29dffdccd5a7f4c8b496c293e87c8668 | spark.deploy.spreadOut | 0.6.1 | None | bb2b9ff37cd2503cc6ea82c5dd395187b0910af0#diff-0e7ae91819fc8f7b47b0f97be7116325 | spark.deploy.defaultCores | 0.9.0 | None | d8bcc8e9a095c1b20dd7a17b6535800d39bff80e#diff-29dffdccd5a7f4c8b496c293e87c8668 | ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT Closes #27668 from beliefer/add-version-to-deploy-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-24 21:39:11 -05:00
<td>0.8.1</td>
</tr>
</table>
### Cluster Managers
Each cluster manager in Spark has additional configuration options. Configurations
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
can be found on the pages for each mode:
#### [YARN](running-on-yarn.html#configuration)
#### [Mesos](running-on-mesos.html#configuration)
#### [Kubernetes](running-on-kubernetes.html#configuration)
#### [Standalone Mode](spark-standalone.html#cluster-launch-scripts)
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
2014-05-28 18:49:54 -04:00
# Environment Variables
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.
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.
The following variables can be set in `spark-env.sh`:
<table class="table">
<tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr>
<tr>
<td><code>JAVA_HOME</code></td>
<td>Location where Java is installed (if it's not on your default <code>PATH</code>).</td>
</tr>
<tr>
<td><code>PYSPARK_PYTHON</code></td>
<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>
</tr>
<tr>
<td><code>PYSPARK_DRIVER_PYTHON</code></td>
<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>
</tr>
<tr>
<td><code>SPARKR_DRIVER_R</code></td>
<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>
</tr>
<tr>
<td><code>SPARK_LOCAL_IP</code></td>
<td>IP address of the machine to bind to.</td>
</tr>
<tr>
<td><code>SPARK_PUBLIC_DNS</code></td>
<td>Hostname your Spark program will advertise to other machines.</td>
</tr>
</table>
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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.
Organize configuration docs This PR improves and organizes the config option page and makes a few other changes to config docs. See a preview here: http://people.apache.org/~pwendell/config-improvements/configuration.html The biggest changes are: 1. The configs for the standalone master/workers were moved to the standalone page and out of the general config doc. 2. SPARK_LOCAL_DIRS was missing from the standalone docs. 3. Expanded discussion of injecting configs with spark-submit, including an example. 4. Config options were organized into the following categories: - Runtime Environment - Shuffle Behavior - Spark UI - Compression and Serialization - Execution Behavior - Networking - Scheduling - Security - Spark Streaming Author: Patrick Wendell <pwendell@gmail.com> Closes #880 from pwendell/config-cleanup and squashes the following commits: 93f56c3 [Patrick Wendell] Feedback from Matei 6f66efc [Patrick Wendell] More feedback 16ae776 [Patrick Wendell] Adding back header section d9c264f [Patrick Wendell] Small fix e0c1728 [Patrick Wendell] Response to Matei's review 27d57db [Patrick Wendell] Reverting changes to index.html (covered in #896) e230ef9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup a374369 [Patrick Wendell] Line wrapping fixes fdff7fc [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup 3289ea4 [Patrick Wendell] Pulling in changes from #856 106ee31 [Patrick Wendell] Small link fix f7e79bc [Patrick Wendell] Re-organizing config options. 54b184d [Patrick Wendell] Adding standalone configs to the standalone page 592e94a [Patrick Wendell] Stash 29b5446 [Patrick Wendell] Better discussion of spark-submit in configuration docs 2d719ef [Patrick Wendell] Small fix 4af9e07 [Patrick Wendell] Adding SPARK_LOCAL_DIRS docs 204b248 [Patrick Wendell] Small fixes
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.
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.
# Configuring Logging
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.
# Overriding configuration directory
To specify a different configuration directory other than the default "SPARK_HOME/conf",
you can set SPARK_CONF_DIR. Spark will use the configuration files (spark-defaults.conf, spark-env.sh, log4j.properties, etc)
from this directory.
# 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 Hadoop versions, but
a common location is inside of `/etc/hadoop/conf`. Some tools create
configurations on-the-fly, but offer a mechanism to download copies of them.
To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/conf/spark-env.sh`
to a location containing the configuration files.
# Custom Hadoop/Hive Configuration
If your Spark application is interacting with Hadoop, Hive, or both, there are probably Hadoop/Hive
configuration files in Spark's classpath.
Multiple running applications might require different Hadoop/Hive client side configurations.
You can copy and modify `hdfs-site.xml`, `core-site.xml`, `yarn-site.xml`, `hive-site.xml` in
Spark's classpath for each application. In a Spark cluster running on YARN, these configuration
files are set cluster-wide, and cannot safely be changed by the application.
The better choice is to use spark hadoop properties in the form of `spark.hadoop.*`, and use
spark hive properties in the form of `spark.hive.*`.
For example, adding configuration "spark.hadoop.abc.def=xyz" represents adding hadoop property "abc.def=xyz",
and adding configuration "spark.hive.abc=xyz" represents adding hive property "hive.abc=xyz".
They can be considered as same as normal spark properties which can be set in `$SPARK_HOME/conf/spark-defaults.conf`
In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For
instance, Spark allows you to simply create an empty conf and set spark/spark hadoop/spark hive properties.
{% highlight scala %}
val conf = new SparkConf().set("spark.hadoop.abc.def", "xyz")
val sc = new SparkContext(conf)
{% endhighlight %}
Also, you can modify or add configurations at runtime:
{% highlight bash %}
./bin/spark-submit \
--name "My app" \
--master local[4] \
--conf spark.eventLog.enabled=false \
--conf "spark.executor.extraJavaOptions=-XX:+PrintGCDetails -XX:+PrintGCTimeStamps" \
--conf spark.hadoop.abc.def=xyz \
--conf spark.hive.abc=xyz
myApp.jar
{% endhighlight %}
# Custom Resource Scheduling and Configuration Overview
GPUs and other accelerators have been widely used for accelerating special workloads, e.g.,
deep learning and signal processing. Spark now supports requesting and scheduling generic resources, such as GPUs, with a few caveats. The current implementation requires that the resource have addresses that can be allocated by the scheduler. It requires your cluster manager to support and be properly configured with the resources.
There are configurations available to request resources for the driver: <code>spark.driver.resource.{resourceName}.amount</code>, request resources for the executor(s): <code>spark.executor.resource.{resourceName}.amount</code> and specify the requirements for each task: <code>spark.task.resource.{resourceName}.amount</code>. The <code>spark.driver.resource.{resourceName}.discoveryScript</code> config is required on YARN, Kubernetes and a client side Driver on Spark Standalone. <code>spark.executor.resource.{resourceName}.discoveryScript</code> config is required for YARN and Kubernetes. Kubernetes also requires <code>spark.driver.resource.{resourceName}.vendor</code> and/or <code>spark.executor.resource.{resourceName}.vendor</code>. See the config descriptions above for more information on each.
Spark will use the configurations specified to first request containers with the corresponding resources from the cluster manager. Once it gets the container, Spark launches an Executor in that container which will discover what resources the container has and the addresses associated with each resource. The Executor will register with the Driver and report back the resources available to that Executor. The Spark scheduler can then schedule tasks to each Executor and assign specific resource addresses based on the resource requirements the user specified. The user can see the resources assigned to a task using the <code>TaskContext.get().resources</code> api. On the driver, the user can see the resources assigned with the SparkContext <code>resources</code> call. It's then up to the user to use the assignedaddresses to do the processing they want or pass those into the ML/AI framework they are using.
See your cluster manager specific page for requirements and details on each of - [YARN](running-on-yarn.html#resource-allocation-and-configuration-overview), [Kubernetes](running-on-kubernetes.html#resource-allocation-and-configuration-overview) and [Standalone Mode](spark-standalone.html#resource-allocation-and-configuration-overview). It is currently not available with Mesos or local mode. And please also note that local-cluster mode with multiple workers is not supported(see Standalone documentation).