This is an automated email from the ASF dual-hosted git repository.

gurwls223 pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git

commit 5e7fa2e713d510aa502b2c83542cf4e57f454fb3
Author: beliefer <[email protected]>
AuthorDate: Mon Mar 16 10:08:07 2020 +0900

    [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 <[email protected]>
    Signed-off-by: HyukjinKwon <[email protected]>
---
 .../org/apache/spark/internal/config/package.scala | 77 +++++++++++++++++++++-
 docs/configuration.md                              | 31 +++++++--
 docs/security.md                                   | 22 +++++--
 3 files changed, 119 insertions(+), 11 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala 
b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 12ae56c..74a2e0a 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -776,47 +776,59 @@ package object config {
 
   // This property sets the root namespace for metrics reporting
   private[spark] val METRICS_NAMESPACE = 
ConfigBuilder("spark.metrics.namespace")
+    .version("2.1.0")
     .stringConf
     .createOptional
 
   private[spark] val METRICS_CONF = ConfigBuilder("spark.metrics.conf")
+    .version("0.8.0")
     .stringConf
     .createOptional
 
   private[spark] val METRICS_EXECUTORMETRICS_SOURCE_ENABLED =
     ConfigBuilder("spark.metrics.executorMetricsSource.enabled")
       .doc("Whether to register the ExecutorMetrics source with the metrics 
system.")
+      .version("3.0.0")
       .booleanConf
       .createWithDefault(true)
 
   private[spark] val METRICS_STATIC_SOURCES_ENABLED =
     ConfigBuilder("spark.metrics.staticSources.enabled")
       .doc("Whether to register static sources with the metrics system.")
+      .version("3.0.0")
       .booleanConf
       .createWithDefault(true)
 
   private[spark] val PYSPARK_DRIVER_PYTHON = 
ConfigBuilder("spark.pyspark.driver.python")
+    .version("2.1.0")
     .stringConf
     .createOptional
 
   private[spark] val PYSPARK_PYTHON = ConfigBuilder("spark.pyspark.python")
+    .version("2.1.0")
     .stringConf
     .createOptional
 
   // To limit how many applications are shown in the History Server summary ui
   private[spark] val HISTORY_UI_MAX_APPS =
-    
ConfigBuilder("spark.history.ui.maxApplications").intConf.createWithDefault(Integer.MAX_VALUE)
+    ConfigBuilder("spark.history.ui.maxApplications")
+      .version("2.0.1")
+      .intConf
+      .createWithDefault(Integer.MAX_VALUE)
 
   private[spark] val IO_ENCRYPTION_ENABLED = 
ConfigBuilder("spark.io.encryption.enabled")
+    .version("2.1.0")
     .booleanConf
     .createWithDefault(false)
 
   private[spark] val IO_ENCRYPTION_KEYGEN_ALGORITHM =
     ConfigBuilder("spark.io.encryption.keygen.algorithm")
+      .version("2.1.0")
       .stringConf
       .createWithDefault("HmacSHA1")
 
   private[spark] val IO_ENCRYPTION_KEY_SIZE_BITS = 
ConfigBuilder("spark.io.encryption.keySizeBits")
+    .version("2.1.0")
     .intConf
     .checkValues(Set(128, 192, 256))
     .createWithDefault(128)
@@ -824,57 +836,68 @@ package object config {
   private[spark] val IO_CRYPTO_CIPHER_TRANSFORMATION =
     ConfigBuilder("spark.io.crypto.cipher.transformation")
       .internal()
+      .version("2.1.0")
       .stringConf
       .createWithDefaultString("AES/CTR/NoPadding")
 
   private[spark] val DRIVER_HOST_ADDRESS = ConfigBuilder("spark.driver.host")
     .doc("Address of driver endpoints.")
+    .version("0.7.0")
     .stringConf
     .createWithDefault(Utils.localCanonicalHostName())
 
   private[spark] val DRIVER_PORT = ConfigBuilder("spark.driver.port")
     .doc("Port of driver endpoints.")
+    .version("0.7.0")
     .intConf
     .createWithDefault(0)
 
   private[spark] val DRIVER_SUPERVISE = ConfigBuilder("spark.driver.supervise")
     .doc("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.")
+    .version("1.3.0")
     .booleanConf
     .createWithDefault(false)
 
   private[spark] val DRIVER_BIND_ADDRESS = 
ConfigBuilder("spark.driver.bindAddress")
     .doc("Address where to bind network listen sockets on the driver.")
+    .version("2.1.0")
     .fallbackConf(DRIVER_HOST_ADDRESS)
 
   private[spark] val BLOCK_MANAGER_PORT = 
ConfigBuilder("spark.blockManager.port")
     .doc("Port to use for the block manager when a more specific setting is 
not provided.")
+    .version("1.1.0")
     .intConf
     .createWithDefault(0)
 
   private[spark] val DRIVER_BLOCK_MANAGER_PORT = 
ConfigBuilder("spark.driver.blockManager.port")
     .doc("Port to use for the block manager on the driver.")
+    .version("2.1.0")
     .fallbackConf(BLOCK_MANAGER_PORT)
 
   private[spark] val IGNORE_CORRUPT_FILES = 
ConfigBuilder("spark.files.ignoreCorruptFiles")
     .doc("Whether to ignore corrupt files. If true, the Spark jobs will 
continue to run when " +
       "encountering corrupted or non-existing files and contents that have 
been read will still " +
       "be returned.")
+    .version("2.1.0")
     .booleanConf
     .createWithDefault(false)
 
   private[spark] val IGNORE_MISSING_FILES = 
ConfigBuilder("spark.files.ignoreMissingFiles")
     .doc("Whether to ignore missing files. If true, the Spark jobs will 
continue to run when " +
       "encountering missing files and the contents that have been read will 
still be returned.")
+    .version("2.4.0")
     .booleanConf
     .createWithDefault(false)
 
   private[spark] val APP_CALLER_CONTEXT = 
ConfigBuilder("spark.log.callerContext")
+    .version("2.2.0")
     .stringConf
     .createOptional
 
   private[spark] val FILES_MAX_PARTITION_BYTES = 
ConfigBuilder("spark.files.maxPartitionBytes")
     .doc("The maximum number of bytes to pack into a single partition when 
reading files.")
+    .version("2.1.0")
     .bytesConf(ByteUnit.BYTE)
     .createWithDefault(128 * 1024 * 1024)
 
@@ -883,6 +906,7 @@ package object config {
       " the same time. This is used when putting multiple files into a 
partition. It's better to" +
       " over estimate, then the partitions with small files will be faster 
than partitions with" +
       " bigger files.")
+    .version("2.1.0")
     .bytesConf(ByteUnit.BYTE)
     .createWithDefault(4 * 1024 * 1024)
 
@@ -890,6 +914,7 @@ package object config {
     ConfigBuilder("spark.hadoopRDD.ignoreEmptySplits")
       .internal()
       .doc("When true, HadoopRDD/NewHadoopRDD will not create partitions for 
empty input splits.")
+      .version("2.3.0")
       .booleanConf
       .createWithDefault(false)
 
@@ -899,6 +924,7 @@ package object config {
         "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.")
+      .version("2.1.2")
       .regexConf
       .createWithDefault("(?i)secret|password|token".r)
 
@@ -907,26 +933,31 @@ package object config {
       .doc("Regex to decide which parts of strings produced by Spark contain 
sensitive " +
         "information. When this regex matches a string part, that string part 
is replaced by a " +
         "dummy value. This is currently used to redact the output of SQL 
explain commands.")
+      .version("2.2.0")
       .regexConf
       .createOptional
 
   private[spark] val AUTH_SECRET =
     ConfigBuilder("spark.authenticate.secret")
+      .version("1.0.0")
       .stringConf
       .createOptional
 
   private[spark] val AUTH_SECRET_BIT_LENGTH =
     ConfigBuilder("spark.authenticate.secretBitLength")
+      .version("1.6.0")
       .intConf
       .createWithDefault(256)
 
   private[spark] val NETWORK_AUTH_ENABLED =
     ConfigBuilder("spark.authenticate")
+      .version("1.0.0")
       .booleanConf
       .createWithDefault(false)
 
   private[spark] val SASL_ENCRYPTION_ENABLED =
     ConfigBuilder("spark.authenticate.enableSaslEncryption")
+      .version("1.4.0")
       .booleanConf
       .createWithDefault(false)
 
@@ -936,6 +967,7 @@ package object config {
         "loaded from this path on both the driver and the executors if 
overrides are not set for " +
         "either entity (see below). File-based secret keys are only allowed 
when using " +
         "Kubernetes.")
+      .version("3.0.0")
       .stringConf
       .createOptional
 
@@ -948,6 +980,7 @@ package object config {
         "be specified for the executors. The fallback configuration allows the 
same path to be " +
         "used for both the driver and the executors when running in cluster 
mode. File-based " +
         "secret keys are only allowed when using Kubernetes.")
+      .version("3.0.0")
       .fallbackConf(AUTH_SECRET_FILE)
 
   private[spark] val AUTH_SECRET_FILE_EXECUTOR =
@@ -959,12 +992,14 @@ package object config {
         "specified for the executors. The fallback configuration allows the 
same path to be " +
         "used for both the driver and the executors when running in cluster 
mode. File-based " +
         "secret keys are only allowed when using Kubernetes.")
+      .version("3.0.0")
       .fallbackConf(AUTH_SECRET_FILE)
 
   private[spark] val BUFFER_WRITE_CHUNK_SIZE =
     ConfigBuilder("spark.buffer.write.chunkSize")
       .internal()
       .doc("The chunk size in bytes during writing out the bytes of 
ChunkedByteBuffer.")
+      .version("2.3.0")
       .bytesConf(ByteUnit.BYTE)
       .checkValue(_ <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH,
         "The chunk size during writing out the bytes of ChunkedByteBuffer 
should" +
@@ -975,6 +1010,7 @@ package object config {
     ConfigBuilder("spark.checkpoint.compress")
       .doc("Whether to compress RDD checkpoints. Generally a good idea. 
Compression will use " +
         "spark.io.compression.codec.")
+      .version("2.2.0")
       .booleanConf
       .createWithDefault(false)
 
@@ -985,6 +1021,7 @@ package object config {
         "Caching preferred locations can relieve query loading to DFS and save 
the query " +
         "time. The drawback is that the cached locations can be possibly 
outdated and " +
         "lose data locality. If this config is not specified, it will not 
cache.")
+      .version("3.0.0")
       .timeConf(TimeUnit.MINUTES)
       .checkValue(_ > 0, "The expire time for caching preferred locations 
cannot be non-positive.")
       .createOptional
@@ -994,12 +1031,14 @@ package object config {
       .doc("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.")
+      .version("2.2.1")
       .bytesConf(ByteUnit.BYTE)
       .createWithDefault(100 * 1024 * 1024)
 
   private[spark] val SHUFFLE_REGISTRATION_TIMEOUT =
     ConfigBuilder("spark.shuffle.registration.timeout")
       .doc("Timeout in milliseconds for registration to the external shuffle 
service.")
+      .version("2.3.0")
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefault(5000)
 
@@ -1007,6 +1046,7 @@ package object config {
     ConfigBuilder("spark.shuffle.registration.maxAttempts")
       .doc("When we fail to register to the external shuffle service, we will 
" +
         "retry for maxAttempts times.")
+      .version("2.3.0")
       .intConf
       .createWithDefault(3)
 
@@ -1017,6 +1057,7 @@ package object config {
         "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 the issue by setting it 
to a lower value.")
+      .version("2.2.1")
       .intConf
       .checkValue(_ > 0, "The max no. of blocks in flight cannot be 
non-positive.")
       .createWithDefault(Int.MaxValue)
@@ -1028,6 +1069,7 @@ package object config {
         "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.")
+      .version("3.0.0")
       .bytesConf(ByteUnit.BYTE)
       // fetch-to-mem is guaranteed to fail if the message is bigger than 2 
GB, so we might
       // as well use fetch-to-disk in that case.  The message includes some 
metadata in addition
@@ -1043,12 +1085,14 @@ package object config {
       .doc("Enable tracking of updatedBlockStatuses in the TaskMetrics. Off by 
default since " +
         "tracking the block statuses can use a lot of memory and its not used 
anywhere within " +
         "spark.")
+      .version("2.3.0")
       .booleanConf
       .createWithDefault(false)
 
   private[spark] val SHUFFLE_IO_PLUGIN_CLASS =
     ConfigBuilder("spark.shuffle.sort.io.plugin.class")
       .doc("Name of the class to use for shuffle IO.")
+      .version("3.0.0")
       .stringConf
       .createWithDefault(classOf[LocalDiskShuffleDataIO].getName)
 
@@ -1057,6 +1101,7 @@ package object config {
       .doc("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.")
+      .version("1.4.0")
       .bytesConf(ByteUnit.KiB)
       .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH 
/ 1024,
         s"The file buffer size must be positive and less than or equal to" +
@@ -1067,6 +1112,7 @@ package object config {
     ConfigBuilder("spark.shuffle.unsafe.file.output.buffer")
       .doc("The file system for this buffer size after each partition " +
         "is written in unsafe shuffle writer. In KiB unless otherwise 
specified.")
+      .version("2.3.0")
       .bytesConf(ByteUnit.KiB)
       .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH 
/ 1024,
         s"The buffer size must be positive and less than or equal to" +
@@ -1076,6 +1122,7 @@ package object config {
   private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE =
     ConfigBuilder("spark.shuffle.spill.diskWriteBufferSize")
       .doc("The buffer size, in bytes, to use when writing the sorted records 
to an on-disk file.")
+      .version("2.3.0")
       .bytesConf(ByteUnit.BYTE)
       .checkValue(v => v > 12 && v <= 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH,
         s"The buffer size must be greater than 12 and less than or equal to " +
@@ -1087,6 +1134,7 @@ package object config {
       .internal()
       .doc("The memory check period is used to determine how often we should 
check whether "
         + "there is a need to request more memory when we try to unroll the 
given block in memory.")
+      .version("2.3.0")
       .longConf
       .createWithDefault(16)
 
@@ -1094,6 +1142,7 @@ package object config {
     ConfigBuilder("spark.storage.unrollMemoryGrowthFactor")
       .internal()
       .doc("Memory to request as a multiple of the size that used to unroll 
the block.")
+      .version("2.3.0")
       .doubleConf
       .createWithDefault(1.5)
 
@@ -1104,12 +1153,14 @@ package object config {
         "where the YARN service does not support schemes that are supported by 
Spark, like http, " +
         "https and ftp, or jars required to be in the local YARN client's 
classpath. Wildcard " +
         "'*' is denoted to download resources for all the schemes.")
+      .version("2.3.0")
       .stringConf
       .toSequence
       .createWithDefault(Nil)
 
   private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners")
     .doc("Class names of listeners to add to SparkContext during 
initialization.")
+    .version("1.3.0")
     .stringConf
     .toSequence
     .createOptional
@@ -1121,6 +1172,7 @@ package object config {
         "By default it's Integer.MAX_VALUE, which means we never force the 
sorter to spill, " +
         "until we reach some limitations, like the max page size limitation 
for the pointer " +
         "array in the sorter.")
+      .version("1.6.0")
       .intConf
       .createWithDefault(Integer.MAX_VALUE)
 
@@ -1130,30 +1182,35 @@ package object config {
       .doc("Multi-thread is used when the number of mappers * shuffle 
partitions is greater than " +
         "or equal to this threshold. Note that the actual parallelism is 
calculated by number of " +
         "mappers * shuffle partitions / this threshold + 1, so this threshold 
should be positive.")
+      .version("2.3.0")
       .intConf
       .checkValue(v => v > 0, "The threshold should be positive.")
       .createWithDefault(10000000)
 
   private[spark] val MAX_RESULT_SIZE = 
ConfigBuilder("spark.driver.maxResultSize")
     .doc("Size limit for results.")
+    .version("1.2.0")
     .bytesConf(ByteUnit.BYTE)
     .createWithDefaultString("1g")
 
   private[spark] val CREDENTIALS_RENEWAL_INTERVAL_RATIO =
     ConfigBuilder("spark.security.credentials.renewalRatio")
       .doc("Ratio of the credential's expiration time when Spark should fetch 
new credentials.")
+      .version("2.4.0")
       .doubleConf
       .createWithDefault(0.75d)
 
   private[spark] val CREDENTIALS_RENEWAL_RETRY_WAIT =
     ConfigBuilder("spark.security.credentials.retryWait")
       .doc("How long to wait before retrying to fetch new credentials after a 
failure.")
+      .version("2.4.0")
       .timeConf(TimeUnit.SECONDS)
       .createWithDefaultString("1h")
 
   private[spark] val SHUFFLE_SORT_INIT_BUFFER_SIZE =
     ConfigBuilder("spark.shuffle.sort.initialBufferSize")
       .internal()
+      .version("2.1.0")
       .bytesConf(ByteUnit.BYTE)
       .checkValue(v => v > 0 && v <= Int.MaxValue,
         s"The buffer size must be greater than 0 and less than or equal to 
${Int.MaxValue}.")
@@ -1163,6 +1220,7 @@ package object config {
     ConfigBuilder("spark.shuffle.compress")
       .doc("Whether to compress shuffle output. Compression will use " +
         "spark.io.compression.codec.")
+      .version("0.6.0")
       .booleanConf
       .createWithDefault(true)
 
@@ -1170,6 +1228,7 @@ package object config {
     ConfigBuilder("spark.shuffle.spill.compress")
       .doc("Whether to compress data spilled during shuffles. Compression will 
use " +
         "spark.io.compression.codec.")
+      .version("0.9.0")
       .booleanConf
       .createWithDefault(true)
 
@@ -1179,6 +1238,7 @@ package object config {
       .doc("The codec used to compress MapStatus, which is generated by 
ShuffleMapTask. " +
         "By default, Spark provides four codecs: lz4, lzf, snappy, and zstd. 
You can also " +
         "use fully qualified class names to specify the codec.")
+      .version("3.0.0")
       .stringConf
       .createWithDefault("zstd")
 
@@ -1187,6 +1247,7 @@ package object config {
       .internal()
       .doc("Initial threshold for the size of a collection before we start 
tracking its " +
         "memory usage.")
+      .version("1.1.1")
       .bytesConf(ByteUnit.BYTE)
       .createWithDefault(5 * 1024 * 1024)
 
@@ -1194,6 +1255,7 @@ package object config {
     ConfigBuilder("spark.shuffle.spill.batchSize")
       .internal()
       .doc("Size of object batches when reading/writing from serializers.")
+      .version("0.9.0")
       .longConf
       .createWithDefault(10000)
 
@@ -1201,34 +1263,40 @@ package object config {
     ConfigBuilder("spark.shuffle.sort.bypassMergeThreshold")
       .doc("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")
+      .version("1.1.1")
       .intConf
       .createWithDefault(200)
 
   private[spark] val SHUFFLE_MANAGER =
     ConfigBuilder("spark.shuffle.manager")
+      .version("1.1.0")
       .stringConf
       .createWithDefault("sort")
 
   private[spark] val SHUFFLE_REDUCE_LOCALITY_ENABLE =
     ConfigBuilder("spark.shuffle.reduceLocality.enabled")
       .doc("Whether to compute locality preferences for reduce tasks")
+      .version("1.5.0")
       .booleanConf
       .createWithDefault(true)
 
   private[spark] val SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST =
     ConfigBuilder("spark.shuffle.mapOutput.minSizeForBroadcast")
       .doc("The size at which we use Broadcast to send the map output statuses 
to the executors.")
+      .version("2.0.0")
       .bytesConf(ByteUnit.BYTE)
       .createWithDefaultString("512k")
 
   private[spark] val SHUFFLE_MAPOUTPUT_DISPATCHER_NUM_THREADS =
     ConfigBuilder("spark.shuffle.mapOutput.dispatcher.numThreads")
+      .version("2.0.0")
       .intConf
       .createWithDefault(8)
 
   private[spark] val SHUFFLE_DETECT_CORRUPT =
     ConfigBuilder("spark.shuffle.detectCorrupt")
       .doc("Whether to detect any corruption in fetched blocks.")
+      .version("2.2.0")
       .booleanConf
       .createWithDefault(true)
 
@@ -1238,18 +1306,21 @@ package object config {
         "by using extra memory to detect early corruption. Any IOException 
thrown will cause " +
         "the task to be retried once and if it fails again with same 
exception, then " +
         "FetchFailedException will be thrown to retry previous stage")
+      .version("3.0.0")
       .booleanConf
       .createWithDefault(false)
 
   private[spark] val SHUFFLE_SYNC =
     ConfigBuilder("spark.shuffle.sync")
       .doc("Whether to force outstanding writes to disk.")
+      .version("0.8.0")
       .booleanConf
       .createWithDefault(false)
 
   private[spark] val SHUFFLE_UNSAFE_FAST_MERGE_ENABLE =
     ConfigBuilder("spark.shuffle.unsafe.fastMergeEnabled")
       .doc("Whether to perform a fast spill merge.")
+      .version("1.4.0")
       .booleanConf
       .createWithDefault(true)
 
@@ -1257,6 +1328,7 @@ package object config {
     ConfigBuilder("spark.shuffle.sort.useRadixSort")
       .doc("Whether to use radix sort for sorting in-memory partition ids. 
Radix sort is much " +
         "faster, but requires additional memory to be reserved memory as 
pointers are added.")
+      .version("2.0.0")
       .booleanConf
       .createWithDefault(true)
 
@@ -1264,6 +1336,7 @@ package object config {
     ConfigBuilder("spark.shuffle.minNumPartitionsToHighlyCompress")
       .internal()
       .doc("Number of partitions to determine if MapStatus should use 
HighlyCompressedMapStatus")
+      .version("2.4.0")
       .intConf
       .checkValue(v => v > 0, "The value should be a positive integer.")
       .createWithDefault(2000)
@@ -1273,6 +1346,7 @@ package object config {
       .doc("Whether to use the old protocol while doing the shuffle block 
fetching. " +
         "It is only enabled while we need the compatibility in the scenario of 
new Spark " +
         "version job fetching shuffle blocks from old version external shuffle 
service.")
+      .version("3.0.0")
       .booleanConf
       .createWithDefault(false)
 
@@ -1282,6 +1356,7 @@ package object config {
         s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled), shuffle " +
         "blocks requested from those block managers which are running on the 
same host are read " +
         "from the disk directly instead of being fetched as remote blocks over 
the network.")
+      .version("3.0.0")
       .booleanConf
       .createWithDefault(true)
 
diff --git a/docs/configuration.md b/docs/configuration.md
index 4f601c8..e4ea25c 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -163,6 +163,7 @@ of the most common options to set are:
     and memory overhead of objects in JVM). Setting a proper limit can protect 
the driver from
     out-of-memory errors.
   </td>
+  <td>1.2.0</td>
 </tr>
 <tr>
   <td><code>spark.driver.memory</code></td>
@@ -322,6 +323,7 @@ of the most common options to set are:
     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>
+  <td>1.3.0</td>
 </tr>
 <tr>
   <td><code>spark.local.dir</code></td>
@@ -368,6 +370,7 @@ of the most common options to set are:
     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>
+  <td>2.2.0</td>
 </tr>
 <tr>
   <td><code>spark.driver.supervise</code></td>
@@ -376,6 +379,7 @@ of the most common options to set are:
     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>
+  <td>1.3.0</td>
 </tr>
 <tr>
   <td><code>spark.driver.log.dfsDir</code></td>
@@ -635,6 +639,7 @@ Apart from these, the following properties are also 
available, and may be useful
     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.
   </td>
+  <td>2.1.2</td>
 </tr>
 <tr>
   <td><code>spark.python.profile</code></td>
@@ -648,6 +653,7 @@ Apart from these, the following properties are also 
available, and may be useful
     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.
   </td>
+  <td></td>
 </tr>
 <tr>
   <td><code>spark.python.profile.dump</code></td>
@@ -757,6 +763,7 @@ Apart from these, the following properties are also 
available, and may be useful
     Python binary executable to use for PySpark in driver.
     (default is <code>spark.pyspark.python</code>)
   </td>
+  <td>2.1.0</td>
 </tr>
 <tr>
   <td><code>spark.pyspark.python</code></td>
@@ -764,6 +771,7 @@ Apart from these, the following properties are also 
available, and may be useful
   <td>
     Python binary executable to use for PySpark in both driver and executors.
   </td>
+  <td>2.1.0</td>
 </tr>
 </table>
 
@@ -801,6 +809,7 @@ Apart from these, the following properties are also 
available, and may be useful
     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>
+  <td>2.2.1</td>
 </tr>
 <tr>
   <td><code>spark.shuffle.compress</code></td>
@@ -809,6 +818,7 @@ Apart from these, the following properties are also 
available, and may be useful
     Whether to compress map output files. Generally a good idea. Compression 
will use
     <code>spark.io.compression.codec</code>.
   </td>
+  <td>0.6.0</td>
 </tr>
 <tr>
   <td><code>spark.shuffle.file.buffer</code></td>
@@ -818,6 +828,7 @@ Apart from these, the following properties are also 
available, and may be useful
     specified. These buffers reduce the number of disk seeks and system calls 
made in creating 
     intermediate shuffle files.
   </td>
+  <td>1.4.0</td>
 </tr>
 <tr>
   <td><code>spark.shuffle.io.maxRetries</code></td>
@@ -912,6 +923,7 @@ Apart from these, the following properties are also 
available, and may be useful
     (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>
+  <td>1.1.1</td>
 </tr>
 <tr>
   <td><code>spark.shuffle.spill.compress</code></td>
@@ -920,6 +932,7 @@ Apart from these, the following properties are also 
available, and may be useful
     Whether to compress data spilled during shuffles. Compression will use
     <code>spark.io.compression.codec</code>.
   </td>
+  <td>0.9.0</td>
 </tr>
 <tr>
   <td><code>spark.shuffle.accurateBlockThreshold</code></td>
@@ -929,6 +942,7 @@ Apart from these, the following properties are also 
available, and may be useful
     accurately recorded. This helps to prevent OOM by avoiding underestimating 
shuffle 
     block size when fetch shuffle blocks.
   </td>
+  <td>2.2.1</td>
 </tr>
 <tr>
   <td><code>spark.shuffle.registration.timeout</code></td>
@@ -936,6 +950,7 @@ Apart from these, the following properties are also 
available, and may be useful
   <td>
     Timeout in milliseconds for registration to the external shuffle service.
   </td>
+  <td>2.3.0</td>
 </tr>
 <tr>
   <td><code>spark.shuffle.registration.maxAttempts</code></td>
@@ -943,6 +958,7 @@ Apart from these, the following properties are also 
available, and may be useful
   <td>
     When we fail to register to the external shuffle service, we will retry 
for maxAttempts times.
   </td>
+  <td>2.3.0</td>
 </tr>
 </table>
 
@@ -1269,7 +1285,8 @@ Apart from these, the following properties are also 
available, and may be useful
   <td>
     Whether to compress RDD checkpoints. Generally a good idea.
     Compression will use <code>spark.io.compression.codec</code>.
-   </td>
+  </td>
+  <td>2.2.0</td>
 </tr>
 <tr>
   <td><code>spark.io.compression.codec</code></td>
@@ -1635,6 +1652,7 @@ Apart from these, the following properties are also 
available, and may be useful
   <td>
     The maximum number of bytes to pack into a single partition when reading 
files.
   </td>
+  <td>2.1.0</td>
 </tr>
 <tr>
   <td><code>spark.files.openCostInBytes</code></td>
@@ -1644,6 +1662,7 @@ Apart from these, the following properties are also 
available, and may be useful
     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>
+  <td>2.1.0</td>
 </tr>
 <tr>
     <td><code>spark.hadoop.cloneConf</code></td>
@@ -1745,7 +1764,7 @@ Apart from these, the following properties are also 
available, and may be useful
   <td>
     Port for all block managers to listen on. These exist on both the driver 
and the executors.
   </td>
-  <td></td>
+  <td>1.1.0</td>
 </tr>
 <tr>
   <td><code>spark.driver.blockManager.port</code></td>
@@ -1754,7 +1773,7 @@ Apart from these, the following properties are also 
available, and may be useful
     Driver-specific port for the block manager to listen on, for cases where 
it cannot use the same
     configuration as executors.
   </td>
-  <td></td>
+  <td>2.1.0</td>
 </tr>
 <tr>
   <td><code>spark.driver.bindAddress</code></td>
@@ -1768,7 +1787,7 @@ Apart from these, the following properties are also 
available, and may be useful
     the different ports used by the driver (RPC, block manager and UI) need to 
be forwarded from the
     container's host.
   </td>
-  <td></td>
+  <td>2.1.0</td>
 </tr>
 <tr>
   <td><code>spark.driver.host</code></td>
@@ -1777,7 +1796,7 @@ Apart from these, the following properties are also 
available, and may be useful
     Hostname or IP address for the driver.
     This is used for communicating with the executors and the standalone 
Master.
   </td>
-  <td></td>
+  <td>0.7.0</td>
 </tr>
 <tr>
   <td><code>spark.driver.port</code></td>
@@ -1786,7 +1805,7 @@ Apart from these, the following properties are also 
available, and may be useful
     Port for the driver to listen on.
     This is used for communicating with the executors and the standalone 
Master.
   </td>
-  <td></td>
+  <td>0.7.0</td>
 </tr>
 <tr>
   <td><code>spark.rpc.io.backLog</code></td>
diff --git a/docs/security.md b/docs/security.md
index a4556e2..5496879 100644
--- a/docs/security.md
+++ b/docs/security.md
@@ -66,11 +66,12 @@ also see their authentication secret. Access control rules 
should be properly se
 Kubernetes admin to ensure that Spark authentication is secure.
 
 <table class="table">
-<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since 
Version</th></tr>
 <tr>
   <td><code>spark.authenticate</code></td>
   <td>false</td>
   <td>Whether Spark authenticates its internal connections.</td>
+  <td>1.0.0</td>
 </tr>
 <tr>
   <td><code>spark.authenticate.secret</code></td>
@@ -78,6 +79,7 @@ Kubernetes admin to ensure that Spark authentication is 
secure.
   <td>
     The secret key used authentication. See above for when this configuration 
should be set.
   </td>
+  <td>1.0.0</td>
 </tr>
 </table>
 
@@ -85,7 +87,7 @@ Alternatively, one can mount authentication secrets using 
files and Kubernetes s
 the user mounts into their pods.
 
 <table class="table">
-<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since 
Version</th></tr>
 <tr>
   <td><code>spark.authenticate.secret.file</code></td>
   <td>None</td>
@@ -94,6 +96,7 @@ the user mounts into their pods.
     contents of the file have been securely generated. This file is loaded on 
both the driver
     and the executors unless other settings override this (see below).
   </td>
+  <td>3.0.0</td>
 </tr>
 <tr>
   <td><code>spark.authenticate.secret.driver.file</code></td>
@@ -106,6 +109,7 @@ the user mounts into their pods.
     and the executors can both use files to load the secret key. Ensure that 
the contents of the file
     on the driver is identical to the contents of the file on the executors.
   </td>
+  <td>3.0.0</td>
 </tr>
 <tr>
   <td><code>spark.authenticate.secret.executor.file</code></td>
@@ -118,6 +122,7 @@ the user mounts into their pods.
     and the executors can both use files to load the secret key. Ensure that 
the contents of the file
     on the driver is identical to the contents of the file on the executors.
   </td>
+  <td>3.0.0</td>
 </tr>
 </table>
 
@@ -138,13 +143,14 @@ is still required when talking to shuffle services from 
Spark versions older tha
 The following table describes the different options available for configuring 
this feature.
 
 <table class="table">
-<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since 
Version</th></tr>
 <tr>
   <td><code>spark.network.crypto.enabled</code></td>
   <td>false</td>
   <td>
     Enable AES-based RPC encryption, including the new authentication protocol 
added in 2.2.0.
   </td>
+  <td>2.2.0</td>
 </tr>
 <tr>
   <td><code>spark.network.crypto.keyLength</code></td>
@@ -152,6 +158,7 @@ The following table describes the different options 
available for configuring th
   <td>
     The length in bits of the encryption key to generate. Valid values are 
128, 192 and 256.
   </td>
+  <td></td>
 </tr>
 <tr>
   <td><code>spark.network.crypto.keyFactoryAlgorithm</code></td>
@@ -160,6 +167,7 @@ The following table describes the different options 
available for configuring th
     The key factory algorithm to use when generating encryption keys. Should 
be one of the
     algorithms supported by the javax.crypto.SecretKeyFactory class in the JRE 
being used.
   </td>
+  <td></td>
 </tr>
 <tr>
   <td><code>spark.network.crypto.config.*</code></td>
@@ -169,6 +177,7 @@ The following table describes the different options 
available for configuring th
     use. The config name should be the name of commons-crypto configuration 
without the
     <code>commons.crypto</code> prefix.
   </td>
+  <td></td>
 </tr>
 <tr>
   <td><code>spark.network.crypto.saslFallback</code></td>
@@ -179,6 +188,7 @@ The following table describes the different options 
available for configuring th
     do not support the internal Spark authentication protocol. On the shuffle 
service side,
     disabling this feature will block older clients from authenticating.
   </td>
+  <td>2.2.0</td>
 </tr>
 <tr>
   <td><code>spark.authenticate.enableSaslEncryption</code></td>
@@ -208,7 +218,7 @@ encrypting output data generated by applications with APIs 
such as `saveAsHadoop
 The following settings cover enabling encryption for data written to disk:
 
 <table class="table">
-<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since 
Version</th></tr>
 <tr>
   <td><code>spark.io.encryption.enabled</code></td>
   <td>false</td>
@@ -216,6 +226,7 @@ The following settings cover enabling encryption for data 
written to disk:
     Enable local disk I/O encryption. Currently supported by all modes except 
Mesos. It's strongly
     recommended that RPC encryption be enabled when using this feature.
   </td>
+  <td>2.1.0</td>
 </tr>
 <tr>
   <td><code>spark.io.encryption.keySizeBits</code></td>
@@ -223,6 +234,7 @@ The following settings cover enabling encryption for data 
written to disk:
   <td>
     IO encryption key size in bits. Supported values are 128, 192 and 256.
   </td>
+  <td>2.1.0</td>
 </tr>
 <tr>
   <td><code>spark.io.encryption.keygen.algorithm</code></td>
@@ -232,6 +244,7 @@ The following settings cover enabling encryption for data 
written to disk:
     described in the KeyGenerator section of the Java Cryptography 
Architecture Standard Algorithm
     Name Documentation.
   </td>
+  <td>2.1.0</td>
 </tr>
 <tr>
   <td><code>spark.io.encryption.commons.config.*</code></td>
@@ -241,6 +254,7 @@ The following settings cover enabling encryption for data 
written to disk:
     use. The config name should be the name of commons-crypto configuration 
without the
     <code>commons.crypto</code> prefix.
   </td>
+  <td>2.1.0</td>
 </tr>
 </table>
 


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to