From bc490f383dc290f5c517fd2399a3211a3a71bc28 Mon Sep 17 00:00:00 2001 From: beliefer Date: Sun, 8 Mar 2020 12:31:57 +0900 Subject: [PATCH] [SPARK-31002][CORE][DOC] Add version information to the configuration of Core MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### 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 Signed-off-by: HyukjinKwon --- .../spark/internal/config/package.scala | 109 +++++++++++++++--- docs/configuration.md | 42 ++++++- 2 files changed, 136 insertions(+), 15 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 23c31a5be8..d4e15621bb 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 @@ -45,6 +45,7 @@ package object config { "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.") + .version("3.0.0") .stringConf .toSequence .createWithDefault(Nil) @@ -55,89 +56,113 @@ package object config { .doc("Path to a file containing the resources allocated to the driver. " + "The file should be formatted as a JSON array of ResourceAllocation objects. " + "Only used internally in standalone mode.") + .version("3.0.0") .stringConf .createOptional private[spark] val DRIVER_CLASS_PATH = - ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH).stringConf.createOptional + ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH) + .version("1.0.0") + .stringConf + .createOptional private[spark] val DRIVER_JAVA_OPTIONS = ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS) .withPrepended(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS) + .version("1.0.0") .stringConf .createOptional private[spark] val DRIVER_LIBRARY_PATH = - ConfigBuilder(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH).stringConf.createOptional + ConfigBuilder(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH) + .version("1.0.0") + .stringConf + .createOptional private[spark] val DRIVER_USER_CLASS_PATH_FIRST = - ConfigBuilder("spark.driver.userClassPathFirst").booleanConf.createWithDefault(false) + ConfigBuilder("spark.driver.userClassPathFirst") + .version("1.3.0") + .booleanConf + .createWithDefault(false) private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores") .doc("Number of cores to use for the driver process, only in cluster mode.") + .version("1.3.0") .intConf .createWithDefault(1) private[spark] val DRIVER_MEMORY = ConfigBuilder(SparkLauncher.DRIVER_MEMORY) .doc("Amount of memory to use for the driver process, in MiB unless otherwise specified.") + .version("1.1.1") .bytesConf(ByteUnit.MiB) .createWithDefaultString("1g") private[spark] val DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.driver.memoryOverhead") .doc("The amount of non-heap memory to be allocated per driver in cluster mode, " + "in MiB unless otherwise specified.") + .version("2.3.0") .bytesConf(ByteUnit.MiB) .createOptional private[spark] val DRIVER_LOG_DFS_DIR = - ConfigBuilder("spark.driver.log.dfsDir").stringConf.createOptional + ConfigBuilder("spark.driver.log.dfsDir").version("3.0.0").stringConf.createOptional private[spark] val DRIVER_LOG_LAYOUT = ConfigBuilder("spark.driver.log.layout") + .version("3.0.0") .stringConf .createOptional private[spark] val DRIVER_LOG_PERSISTTODFS = ConfigBuilder("spark.driver.log.persistToDfs.enabled") + .version("3.0.0") .booleanConf .createWithDefault(false) private[spark] val DRIVER_LOG_ALLOW_EC = ConfigBuilder("spark.driver.log.allowErasureCoding") + .version("3.0.0") .booleanConf .createWithDefault(false) private[spark] val EVENT_LOG_ENABLED = ConfigBuilder("spark.eventLog.enabled") + .version("1.0.0") .booleanConf .createWithDefault(false) private[spark] val EVENT_LOG_DIR = ConfigBuilder("spark.eventLog.dir") + .version("1.0.0") .stringConf .createWithDefault(EventLoggingListener.DEFAULT_LOG_DIR) private[spark] val EVENT_LOG_COMPRESS = ConfigBuilder("spark.eventLog.compress") + .version("1.0.0") .booleanConf .createWithDefault(false) private[spark] val EVENT_LOG_BLOCK_UPDATES = ConfigBuilder("spark.eventLog.logBlockUpdates.enabled") + .version("2.3.0") .booleanConf .createWithDefault(false) private[spark] val EVENT_LOG_ALLOW_EC = ConfigBuilder("spark.eventLog.erasureCoding.enabled") + .version("3.0.0") .booleanConf .createWithDefault(false) private[spark] val EVENT_LOG_TESTING = ConfigBuilder("spark.eventLog.testing") .internal() + .version("1.0.1") .booleanConf .createWithDefault(false) private[spark] val EVENT_LOG_OUTPUT_BUFFER_SIZE = ConfigBuilder("spark.eventLog.buffer.kb") .doc("Buffer size to use when writing to output streams, in KiB unless otherwise specified.") + .version("1.0.0") .bytesConf(ByteUnit.KiB) .createWithDefaultString("100k") @@ -145,6 +170,7 @@ package object config { ConfigBuilder("spark.eventLog.logStageExecutorMetrics") .doc("Whether to write per-stage peaks of executor metrics (for each executor) " + "to the event log.") + .version("3.0.0") .booleanConf .createWithDefault(false) @@ -153,6 +179,7 @@ package object config { .doc("Names of supported young generation garbage collector. A name usually is " + " the return of GarbageCollectorMXBean.getName. The built-in young generation garbage " + s"collectors are ${GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS}") + .version("3.0.0") .stringConf .toSequence .createWithDefault(GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS) @@ -162,20 +189,28 @@ package object config { .doc("Names of supported old generation garbage collector. A name usually is " + "the return of GarbageCollectorMXBean.getName. The built-in old generation garbage " + s"collectors are ${GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS}") + .version("3.0.0") .stringConf .toSequence .createWithDefault(GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS) private[spark] val EVENT_LOG_OVERWRITE = - ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) + ConfigBuilder("spark.eventLog.overwrite") + .version("1.0.0") + .booleanConf + .createWithDefault(false) private[spark] val EVENT_LOG_CALLSITE_LONG_FORM = - ConfigBuilder("spark.eventLog.longForm.enabled").booleanConf.createWithDefault(false) + ConfigBuilder("spark.eventLog.longForm.enabled") + .version("2.4.0") + .booleanConf + .createWithDefault(false) private[spark] val EVENT_LOG_ENABLE_ROLLING = ConfigBuilder("spark.eventLog.rolling.enabled") .doc("Whether rolling over event log files is enabled. If set to true, it cuts down " + "each event log file to the configured size.") + .version("3.0.0") .booleanConf .createWithDefault(false) @@ -183,35 +218,46 @@ package object config { ConfigBuilder("spark.eventLog.rolling.maxFileSize") .doc(s"When ${EVENT_LOG_ENABLE_ROLLING.key}=true, specifies the max size of event log file" + " to be rolled over.") + .version("3.0.0") .bytesConf(ByteUnit.BYTE) .checkValue(_ >= ByteUnit.MiB.toBytes(10), "Max file size of event log should be " + "configured to be at least 10 MiB.") .createWithDefaultString("128m") private[spark] val EXECUTOR_ID = - ConfigBuilder("spark.executor.id").stringConf.createOptional + ConfigBuilder("spark.executor.id").version("1.2.0").stringConf.createOptional private[spark] val EXECUTOR_CLASS_PATH = - ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional + ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH) + .version("1.0.0") + .stringConf + .createOptional private[spark] val EXECUTOR_HEARTBEAT_DROP_ZERO_ACCUMULATOR_UPDATES = ConfigBuilder("spark.executor.heartbeat.dropZeroAccumulatorUpdates") .internal() + .version("3.0.0") .booleanConf .createWithDefault(true) private[spark] val EXECUTOR_HEARTBEAT_INTERVAL = ConfigBuilder("spark.executor.heartbeatInterval") + .version("1.1.0") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("10s") private[spark] val EXECUTOR_HEARTBEAT_MAX_FAILURES = - ConfigBuilder("spark.executor.heartbeat.maxFailures").internal().intConf.createWithDefault(60) + ConfigBuilder("spark.executor.heartbeat.maxFailures") + .internal() + .version("1.6.2") + .intConf + .createWithDefault(60) private[spark] val EXECUTOR_PROCESS_TREE_METRICS_ENABLED = ConfigBuilder("spark.executor.processTreeMetrics.enabled") .doc("Whether to collect process tree metrics (from the /proc filesystem) when collecting " + "executor metrics.") + .version("3.0.0") .booleanConf .createWithDefault(false) @@ -220,33 +266,44 @@ package object config { .doc("How often to collect executor metrics (in milliseconds). " + "If 0, the polling is done on executor heartbeats. " + "If positive, the polling is done at this interval.") + .version("3.0.0") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("0") private[spark] val EXECUTOR_JAVA_OPTIONS = ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS) .withPrepended(SparkLauncher.EXECUTOR_DEFAULT_JAVA_OPTIONS) + .version("1.0.0") .stringConf .createOptional private[spark] val EXECUTOR_LIBRARY_PATH = - ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH).stringConf.createOptional + ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH) + .version("1.0.0") + .stringConf + .createOptional private[spark] val EXECUTOR_USER_CLASS_PATH_FIRST = - ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.createWithDefault(false) + ConfigBuilder("spark.executor.userClassPathFirst") + .version("1.3.0") + .booleanConf + .createWithDefault(false) private[spark] val EXECUTOR_CORES = ConfigBuilder(SparkLauncher.EXECUTOR_CORES) + .version("1.0.0") .intConf .createWithDefault(1) private[spark] val EXECUTOR_MEMORY = ConfigBuilder(SparkLauncher.EXECUTOR_MEMORY) .doc("Amount of memory to use per executor process, in MiB unless otherwise specified.") + .version("0.7.0") .bytesConf(ByteUnit.MiB) .createWithDefaultString("1g") private[spark] val EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.executor.memoryOverhead") .doc("The amount of non-heap memory to be allocated per executor in cluster mode, " + "in MiB unless otherwise specified.") + .version("2.3.0") .bytesConf(ByteUnit.MiB) .createOptional @@ -256,12 +313,14 @@ package object config { "the cluster (not from each machine). If not set, the default will be " + "`spark.deploy.defaultCores` on Spark's standalone cluster manager, or infinite " + "(all available cores) on Mesos.") + .version("0.6.0") .intConf .createOptional private[spark] val MEMORY_OFFHEAP_ENABLED = ConfigBuilder("spark.memory.offHeap.enabled") .doc("If true, Spark will attempt to use off-heap memory for certain operations. " + "If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive.") + .version("1.6.0") .withAlternative("spark.unsafe.offHeap") .booleanConf .createWithDefault(false) @@ -272,6 +331,7 @@ package object config { "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 spark.memory.offHeap.enabled=true.") + .version("1.6.0") .bytesConf(ByteUnit.BYTE) .checkValue(_ >= 0, "The off-heap memory size must not be negative") .createWithDefault(0) @@ -281,6 +341,7 @@ package object config { "size of the region set aside by spark.memory.fraction. 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. ") + .version("1.6.0") .doubleConf .checkValue(v => v >= 0.0 && v < 1.0, "Storage fraction must be in [0,1)") .createWithDefault(0.5) @@ -291,16 +352,19 @@ package object config { "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. ") + .version("1.6.0") .doubleConf .createWithDefault(0.6) private[spark] val STORAGE_SAFETY_FRACTION = ConfigBuilder("spark.storage.safetyFraction") + .version("1.1.0") .doubleConf .createWithDefault(0.9) private[spark] val STORAGE_UNROLL_MEMORY_THRESHOLD = ConfigBuilder("spark.storage.unrollMemoryThreshold") .doc("Initial memory to request before unrolling any block") + .version("1.1.0") .longConf .createWithDefault(1024 * 1024) @@ -310,6 +374,7 @@ package object config { "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") + .version("2.2.0") .booleanConf .createWithDefault(false) @@ -320,41 +385,54 @@ package object config { "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.") + .version("0.9.2") .bytesConf(ByteUnit.BYTE) .createWithDefaultString("2m") private[spark] val STORAGE_REPLICATION_POLICY = ConfigBuilder("spark.storage.replication.policy") + .version("2.1.0") .stringConf .createWithDefaultString(classOf[RandomBlockReplicationPolicy].getName) private[spark] val STORAGE_REPLICATION_TOPOLOGY_MAPPER = ConfigBuilder("spark.storage.replication.topologyMapper") + .version("2.1.0") .stringConf .createWithDefaultString(classOf[DefaultTopologyMapper].getName) private[spark] val STORAGE_CACHED_PEERS_TTL = ConfigBuilder("spark.storage.cachedPeersTtl") - .intConf.createWithDefault(60 * 1000) + .version("1.1.1") + .intConf + .createWithDefault(60 * 1000) private[spark] val STORAGE_MAX_REPLICATION_FAILURE = ConfigBuilder("spark.storage.maxReplicationFailures") - .intConf.createWithDefault(1) + .version("1.1.1") + .intConf + .createWithDefault(1) private[spark] val STORAGE_REPLICATION_TOPOLOGY_FILE = - ConfigBuilder("spark.storage.replication.topologyFile").stringConf.createOptional + ConfigBuilder("spark.storage.replication.topologyFile") + .version("2.1.0") + .stringConf + .createOptional private[spark] val STORAGE_EXCEPTION_PIN_LEAK = ConfigBuilder("spark.storage.exceptionOnPinLeak") + .version("1.6.2") .booleanConf .createWithDefault(false) private[spark] val STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL = ConfigBuilder("spark.storage.blockManagerTimeoutIntervalMs") + .version("0.7.3") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("60s") private[spark] val STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT = ConfigBuilder("spark.storage.blockManagerSlaveTimeoutMs") + .version("0.7.0") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString(Network.NETWORK_TIMEOUT.defaultValueString) @@ -362,6 +440,7 @@ package object config { ConfigBuilder("spark.storage.cleanupFilesAfterExecutorExit") .doc("Whether or not cleanup the files not served by the external shuffle service " + "on executor exits.") + .version("2.4.0") .booleanConf .createWithDefault(true) @@ -369,6 +448,7 @@ package object config { ConfigBuilder("spark.diskStore.subDirectories") .doc("Number of subdirectories inside each path listed in spark.local.dir for " + "hashing Block files into.") + .version("0.6.0") .intConf .checkValue(_ > 0, "The number of subdirectories must be positive.") .createWithDefault(64) @@ -377,6 +457,7 @@ package object config { ConfigBuilder("spark.block.failures.beforeLocationRefresh") .doc("Max number of failures before this block manager refreshes " + "the block locations from the driver.") + .version("2.0.0") .intConf .createWithDefault(5) diff --git a/docs/configuration.md b/docs/configuration.md index b336289741..7afb715bc7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -150,6 +150,7 @@ of the most common options to set are: Number of cores to use for the driver process, only in cluster mode. + 1.3.0 spark.driver.maxResultSize @@ -176,6 +177,7 @@ of the most common options to set are: Instead, please set this through the --driver-memory command line option or in your default properties file. + 1.1.1 spark.driver.memoryOverhead @@ -192,6 +194,7 @@ of the most common options to set are: driver is determined by the sum of spark.driver.memoryOverhead and spark.driver.memory. + 2.3.0 spark.driver.resource.{resourceName}.amount @@ -234,6 +237,7 @@ of the most common options to set are: returns the resource information for that resource. It tries the discovery script last if none of the plugins return information for that resource. + 3.0.0 spark.executor.memory @@ -242,6 +246,7 @@ of the most common options to set are: 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. 512m, 2g). + 0.7.0 spark.executor.pyspark.memory @@ -276,6 +281,7 @@ of the most common options to set are: spark.executor.memory, spark.memory.offHeap.size and spark.executor.pyspark.memory. + 2.3.0 spark.executor.resource.{resourceName}.amount @@ -384,6 +390,7 @@ of the most common options to set are: spark.history.fs.driverlog.cleaner.enabled is true and, if they are older than max age configured by setting spark.history.fs.driverlog.cleaner.maxAge. + 3.0.0 spark.driver.log.persistToDfs.enabled @@ -394,6 +401,7 @@ of the most common options to set are: will not be persisted. Additionally, enable the cleaner by setting spark.history.fs.driverlog.cleaner.enabled to true in Spark History Server. + 3.0.0 spark.driver.log.layout @@ -403,6 +411,7 @@ of the most common options to set are: it uses the layout for the first appender defined in log4j.properties. If that is also not configured, driver logs use the default layout. + 3.0.0 spark.driver.log.allowErasureCoding @@ -413,6 +422,7 @@ of the most common options to set are: 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. + 3.0.0 @@ -433,6 +443,7 @@ Apart from these, the following properties are also available, and may be useful Instead, please set this through the --driver-class-path command line option or in your default properties file. + 1.0.0 spark.driver.defaultJavaOptions @@ -470,6 +481,7 @@ Apart from these, the following properties are also available, and may be useful spark.driver.defaultJavaOptions will be prepended to this configuration. + 1.0.0 spark.driver.extraLibraryPath @@ -482,6 +494,7 @@ Apart from these, the following properties are also available, and may be useful Instead, please set this through the --driver-library-path command line option or in your default properties file. + 1.0.0 spark.driver.userClassPathFirst @@ -493,6 +506,7 @@ Apart from these, the following properties are also available, and may be useful This is used in cluster mode only. + 1.3.0 spark.executor.extraClassPath @@ -502,6 +516,7 @@ Apart from these, the following properties are also available, and may be useful backwards-compatibility with older versions of Spark. Users typically should not need to set this option. + 1.0.0 spark.executor.defaultJavaOptions @@ -539,6 +554,7 @@ Apart from these, the following properties are also available, and may be useful spark.executor.defaultJavaOptions will be prepended to this configuration. + 1.0.0 spark.executor.extraLibraryPath @@ -546,6 +562,7 @@ Apart from these, the following properties are also available, and may be useful Set a special library path to use when launching executor JVM's. + 1.0.0 spark.executor.logs.rolling.maxRetainedFiles @@ -600,6 +617,7 @@ Apart from these, the following properties are also available, and may be useful (Experimental) Same functionality as spark.driver.userClassPathFirst, but applied to executor instances. + 1.3.0 spark.executorEnv.[EnvironmentVariableName] @@ -937,6 +955,7 @@ Apart from these, the following properties are also available, and may be useful Whether to log events for every block update, if spark.eventLog.enabled is true. *Warning*: This will increase the size of the event log considerably. + 2.3.0 spark.eventLog.longForm.enabled @@ -944,6 +963,7 @@ Apart from these, the following properties are also available, and may be useful If true, use the long form of call sites in the event log. Otherwise use the short form. + 2.4.0 spark.eventLog.compress @@ -951,6 +971,7 @@ Apart from these, the following properties are also available, and may be useful Whether to compress logged events, if spark.eventLog.enabled is true. + 1.0.0 spark.eventLog.compression.codec @@ -970,6 +991,7 @@ Apart from these, the following properties are also available, and may be useful Note that even if this is true, Spark will still not force the file to use erasure coding, it will simply use filesystem defaults. + 3.0.0 spark.eventLog.dir @@ -980,6 +1002,7 @@ Apart from these, the following properties are also available, and may be useful 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. + 1.0.0 spark.eventLog.enabled @@ -988,6 +1011,7 @@ Apart from these, the following properties are also available, and may be useful Whether to log Spark events, useful for reconstructing the Web UI after the application has finished. + 1.0.0 spark.eventLog.overwrite @@ -995,6 +1019,7 @@ Apart from these, the following properties are also available, and may be useful Whether to overwrite any existing files. + 1.0.0 spark.eventLog.buffer.kb @@ -1002,6 +1027,7 @@ Apart from these, the following properties are also available, and may be useful Buffer size to use when writing to output streams, in KiB unless otherwise specified. + 1.0.0 spark.eventLog.rolling.enabled @@ -1010,6 +1036,7 @@ Apart from these, the following properties are also available, and may be useful Whether rolling over event log files is enabled. If set to true, it cuts down each event log file to the configured size. + 3.0.0 spark.eventLog.rolling.maxFileSize @@ -1017,6 +1044,7 @@ Apart from these, the following properties are also available, and may be useful When spark.eventLog.rolling.enabled=true, specifies the max size of event log file before it's rolled over. + 3.0.0 spark.ui.dagGraph.retainedRootRDDs @@ -1413,6 +1441,7 @@ Apart from these, the following properties are also available, and may be useful garbage collection when increasing this value, see this description. + 1.6.0 spark.memory.storageFraction @@ -1424,6 +1453,7 @@ Apart from these, the following properties are also available, and may be useful Leaving this at the default value is recommended. For more detail, see this description. + 1.6.0 spark.memory.offHeap.enabled @@ -1432,6 +1462,7 @@ Apart from these, the following properties are also available, and may be useful If true, Spark will attempt to use off-heap memory for certain operations. If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive. + 1.6.0 spark.memory.offHeap.size @@ -1442,6 +1473,7 @@ Apart from these, the following properties are also available, and may be useful 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 spark.memory.offHeap.enabled=true. + 1.6.0 spark.storage.replication.proactive @@ -1451,6 +1483,7 @@ Apart from these, the following properties are also available, and may be useful 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. + 2.2.0 spark.cleaner.periodicGC.interval @@ -1528,6 +1561,7 @@ Apart from these, the following properties are also available, and may be useful In standalone and Mesos coarse-grained modes, for more detail, see this description. + 1.0.0 spark.default.parallelism @@ -1553,6 +1587,7 @@ Apart from these, the following properties are also available, and may be useful 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.network.timeout + 1.1.0 spark.files.fetchTimeout @@ -1625,6 +1660,7 @@ Apart from these, the following properties are also available, and may be useful 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. + 0.9.2 spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version @@ -1640,7 +1676,7 @@ Apart from these, the following properties are also available, and may be useful ### Executor Metrics - + @@ -1651,6 +1687,7 @@ Apart from these, the following properties are also available, and may be useful and this is always done; this configuration is only to determine if aggregated metric peaks are written to the event log. + @@ -1661,6 +1698,7 @@ Apart from these, the following properties are also available, and may be useful Note: The process tree metrics are collected only if the /proc filesystem exists. + @@ -1671,6 +1709,7 @@ Apart from these, the following properties are also available, and may be useful specified by spark.executor.heartbeatInterval). If positive, the polling is done at this interval. +
Property NameDefaultMeaning
Property NameDefaultMeaningSince Version
spark.eventLog.logStageExecutorMetrics false3.0.0
spark.executor.processTreeMetrics.enabled false3.0.0
spark.executor.metrics.pollingInterval 03.0.0
@@ -1854,6 +1893,7 @@ Apart from these, the following properties are also available, and may be useful spark.deploy.defaultCores on Spark's standalone cluster manager, or infinite (all available cores) on Mesos. + 0.6.0 spark.locality.wait