[SPARK-30841][SQL][DOC] Add version information to the configuration of SQL

### What changes were proposed in this pull request?
Add version information to the configuration of Spark SQL.
Note: Because SQLConf has a lot of configuration items, I split the items into two PR. Another PR will follows this PR.

I sorted out some information show below.

Item name | Since version | JIRA ID | Commit ID | Note
-- | -- | -- | -- | --
spark.sql.analyzer.maxIterations | 3.0.0 | SPARK-30138 | c2f29d5ea58eb4565cc5602937d6d0bb75558513#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.excludedRules | 2.4.0 | SPARK-24802 | 434319e73f8cb6e080671bdde42a72228bd814ef#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.maxIterations | 2.0.0 | SPARK-14677 | f4be0946af219379fb2476e6f80b2e50463adeb2#diff-32bb9518401c0948c5ea19377b5069ab |  
spark.sql.optimizer.inSetConversionThreshold | 2.0.0 | SPARK-14796 | 3647120a5a879edf3a96a5fd68fb7aa849ad57ef#diff-32bb9518401c0948c5ea19377b5069ab |  
spark.sql.optimizer.inSetSwitchThreshold | 3.0.0 | SPARK-26205 | 0c23a39384b7ae5fb4aeb4f7f6fe72007b84bbd2#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.planChangeLog.level | 3.0.0 | SPARK-25415 | 8b702e1e0aba1d3e4b0aa582f20cf99f80a44a09#diff-9a6b543db706f1a90f790783d6930a13 | This configuration does not exist in branch-2.4 branch, but from the branch-3.0 git log, it is found that the version number of the pom.xml file is 2.4.0-SNAPSHOT
spark.sql.optimizer.planChangeLog.rules | 3.0.0 | SPARK-25415 | 8b702e1e0aba1d3e4b0aa582f20cf99f80a44a09#diff-9a6b543db706f1a90f790783d6930a13 | This configuration does not exist in branch-2.4 branch, but from the branch-3.0 git log, it is found that the version number of the pom.xml file is 2.4.0-SNAPSHOT
spark.sql.optimizer.planChangeLog.batches | 3.0.0 | SPARK-27088 | 074533334d01afdd7862a1ac6c5a7a672bcce3f8#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.dynamicPartitionPruning.enabled | 3.0.0 | SPARK-11150 | a7a3935c97d1fe6060cae42bbc9229c087b648ab#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.dynamicPartitionPruning.useStats | 3.0.0 | SPARK-11150 | a7a3935c97d1fe6060cae42bbc9229c087b648ab#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.dynamicPartitionPruning.fallbackFilterRatio | 3.0.0 | SPARK-11150 | a7a3935c97d1fe6060cae42bbc9229c087b648ab#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly | 3.0.0 | SPARK-30528 | 59a13c9b7bc3b3aa5b5bc30a60344f849c0f8012#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.inMemoryColumnarStorage.compressed | 1.0.1 | SPARK-2631 | 86534d0f5255362618c05a07b0171ec35c915822#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.inMemoryColumnarStorage.batchSize | 1.1.1 | SPARK-2650 | 779d1eb26d0f031791e93c908d51a59c3b422a55#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.inMemoryColumnarStorage.partitionPruning | 1.2.0 | SPARK-2961 | 248067adbe90f93c7d5e23aa61b3072dfdf48a8a#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.inMemoryTableScanStatistics.enable | 3.0.0 | SPARK-28257 | 42b80ae128ab1aa8a87c1376fe88e2cde52e6e4f#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.inMemoryColumnarStorage.enableVectorizedReader | 2.3.1 | SPARK-23312 | e5e9f9a430c827669ecfe9d5c13cc555fc89c980#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.columnVector.offheap.enabled | 2.3.0 | SPARK-20101 | 572af5027e45ca96e0d283a8bf7c84dcf476f9bc#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.join.preferSortMergeJoin | 2.0.0 | SPARK-13977 | 9c23c818ca0175c8f2a4a66eac261ec251d27c97#diff-32bb9518401c0948c5ea19377b5069ab |  
spark.sql.sort.enableRadixSort | 2.0.0 | SPARK-14724 | e2b5647ab92eb478b3f7b36a0ce6faf83e24c0e5#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.autoBroadcastJoinThreshold | 1.1.0 | SPARK-2393 | c7db274be79f448fda566208946cb50958ea9b1a#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.limit.scaleUpFactor | 2.1.1 | SPARK-19944 | 80ebca62cbdb7d5c8606e95a944164ab1a943694#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.hive.advancedPartitionPredicatePushdown.enabled | 2.3.0 | SPARK-20331 | d8cada8d1d3fce979a4bc1f9879593206722a3b9#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.shuffle.partitions | 1.1.0 | SPARK-1508 | 08ed9ad81397b71206c4dc903bfb94b6105691ed#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.adaptive.enabled | 1.6.0 | SPARK-9858 and SPARK-9859 and SPARK-9861 | d728d5c98658c44ed2949b55d36edeaa46f8c980#diff-41ef65b9ef5b518f77e2a03559893f4d |
spark.sql.adaptive.forceApply | 3.0.0 | SPARK-30719 | b29cb1a82b1a1facf1dd040025db93d998dad4cd#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.shuffle.reducePostShufflePartitions | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.shuffle.minNumPostShufflePartitions | 3.0.0 | SPARK-9853 | 8616109061efc5b23b24bb9ec4a3c0f2745903c1#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.shuffle.targetPostShuffleInputSize | 1.6.0 | SPARK-9858 and SPARK-9859 and SPARK-9861 | d728d5c98658c44ed2949b55d36edeaa46f8c980#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.adaptive.shuffle.maxNumPostShufflePartitions | 3.0.0 | SPARK-9853 | 8616109061efc5b23b24bb9ec4a3c0f2745903c1#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.shuffle.localShuffleReader.enabled | 3.0.0 | SPARK-29893 | 6e581cf164c3a2930966b270ac1406dc1195c942#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.skewedJoinOptimization.enabled | 3.0.0 | SPARK-30812 | b76bc0b1b8b2abd00a84f805af90ca4c5925faaa#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.skewedJoinOptimization.skewedPartitionFactor | 3.0.0 | SPARK-30812 | 5b36cdbbfef147e93b35eaa4f8e0bea9690b6d06#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin | 3.0.0 | SPARK-9853 and SPARK-29002 | 8616109061efc5b23b24bb9ec4a3c0f2745903c1#diff-9a6b543db706f1a90f790783d6930a13 and b2f06608b785f577999318c00f2c315f39d90889#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.subexpressionElimination.enabled | 1.6.0 | SPARK-10371 | f38509a763816f43a224653fe65e4645894c9fc4#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.caseSensitive | 1.4.0 | SPARK-4699 | 21bd7222e55b9cf684c072141998a0623a69f514#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.constraintPropagation.enabled | 2.2.0 | SPARK-19846 | e011004bedca47be998a0c14fe22a6f9bb5090cd#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parser.escapedStringLiterals | 2.2.1 | SPARK-20399 | 3d1908fd58fd9b1970cbffebdb731bfe4c776ad9#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.sources.fileCompressionFactor | 2.3.1 | SPARK-22790 | 0fc5533e53ad03eb67590ddd231f40c2713150c3#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parquet.mergeSchema | 1.5.0 | SPARK-8690 | 246265f2bb056d5e9011d3331b809471a24ff8d7#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.respectSummaryFiles | 1.5.0 | SPARK-8838 | 6175d6cfe795fbd88e3ee713fac375038a3993a8#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.binaryAsString | 1.1.1 | SPARK-2927 | de501e169f24e4573747aec85b7651c98633c028#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.int96AsTimestamp | 1.3.0 | SPARK-4987 | 67d52207b5cf2df37ca70daff2a160117510f55e#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.int96TimestampConversion | 2.3.0 | SPARK-12297 | acf7ef3154e094875fa89f30a78ab111b267db91#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parquet.outputTimestampType | 2.3.0 | SPARK-10365 | 21a7bfd5c324e6c82152229f1394f26afeae771c#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parquet.compression.codec | 1.1.1 | SPARK-3131 | 3a9d874d7a46ab8b015631d91ba479d9a0ba827f#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.filterPushdown | 1.2.0 | SPARK-4391 | 576688aa2a19bd4ba239a2b93af7947f983e5124#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.filterPushdown.date | 2.4.0 | SPARK-23727 | b02e76cbffe9e589b7a4e60f91250ca12a4420b2#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parquet.filterPushdown.timestamp | 2.4.0 | SPARK-24718 | 43e4e851b642bbee535d22e1b9e72ec6b99f6ed4#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.parquet.filterPushdown.decimal | 2.4.0 | SPARK-24549 | 9549a2814951f9ba969955d78ac4bd2240f85989#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.parquet.filterPushdown.string.startsWith | 2.4.0 | SPARK-24638 | 03545ce6de08bd0ad685c5f59b73bc22dfc40887#diff-9a6b543db706f1a90f790783d6930a13 | 
spark.sql.parquet.pushdown.inFilterThreshold | 2.4.0 | SPARK-17091 | e1de34113e057707dfc5ff54a8109b3ec7c16dfb#diff-9a6b543db706f1a90f790783d6930a13 |  
spark.sql.parquet.writeLegacyFormat | 1.6.0 | SPARK-10400 | 01cd688f5245cbb752863100b399b525b31c3510#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.output.committer.class | 1.5.0 | SPARK-8139 | 111d6b9b8a584b962b6ae80c7aa8c45845ce0099#diff-41ef65b9ef5b518f77e2a03559893f4d |  
spark.sql.parquet.enableVectorizedReader | 2.0.0 | SPARK-13486 | 2b2c8c33236677c916541f956f7b94bba014a9ce#diff-32bb9518401c0948c5ea19377b5069ab |
spark.sql.parquet.recordLevelFilter.enabled | 2.3.0 | SPARK-17310 | 673c67046598d33b9ecf864024ca7a937c1998d6#diff-9a6b543db706f1a90f790783d6930a13 |
spark.sql.parquet.columnarReaderBatchSize | 2.4.0 | SPARK-23188 | cc41245fa3f954f961541bf4b4275c28473042b8#diff-9a6b543db706f1a90f790783d6930a13 |  

### 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 #27691 from beliefer/add-version-to-sql-config-part-one.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
This commit is contained in:
beliefer 2020-02-27 10:58:44 +09:00 committed by HyukjinKwon
parent c2857501d5
commit 825d3dc11b

View file

@ -175,6 +175,7 @@ object SQLConf {
val ANALYZER_MAX_ITERATIONS = buildConf("spark.sql.analyzer.maxIterations")
.internal()
.doc("The max number of iterations the analyzer runs.")
.version("3.0.0")
.intConf
.createWithDefault(100)
@ -183,12 +184,14 @@ object SQLConf {
"specified by their rule names and separated by comma. It is not guaranteed that all the " +
"rules in this configuration will eventually be excluded, as some rules are necessary " +
"for correctness. The optimizer will log the rules that have indeed been excluded.")
.version("2.4.0")
.stringConf
.createOptional
val OPTIMIZER_MAX_ITERATIONS = buildConf("spark.sql.optimizer.maxIterations")
.internal()
.doc("The max number of iterations the optimizer runs.")
.version("2.0.0")
.intConf
.createWithDefault(100)
@ -196,6 +199,7 @@ object SQLConf {
buildConf("spark.sql.optimizer.inSetConversionThreshold")
.internal()
.doc("The threshold of set size for InSet conversion.")
.version("2.0.0")
.intConf
.createWithDefault(10)
@ -204,6 +208,7 @@ object SQLConf {
.internal()
.doc("Configures the max set size in InSet for which Spark will generate code with " +
"switch statements. This is applicable only to bytes, shorts, ints, dates.")
.version("3.0.0")
.intConf
.checkValue(threshold => threshold >= 0 && threshold <= 600, "The max set size " +
"for using switch statements in InSet must be non-negative and less than or equal to 600")
@ -214,6 +219,7 @@ object SQLConf {
.doc("Configures the log level for logging the change from the original plan to the new " +
"plan after a rule or batch is applied. The value can be 'trace', 'debug', 'info', " +
"'warn', or 'error'. The default log level is 'trace'.")
.version("3.0.0")
.stringConf
.transform(_.toUpperCase(Locale.ROOT))
.checkValue(logLevel => Set("TRACE", "DEBUG", "INFO", "WARN", "ERROR").contains(logLevel),
@ -225,6 +231,7 @@ object SQLConf {
.internal()
.doc("Configures a list of rules to be logged in the optimizer, in which the rules are " +
"specified by their rule names and separated by comma.")
.version("3.0.0")
.stringConf
.createOptional
@ -232,12 +239,14 @@ object SQLConf {
.internal()
.doc("Configures a list of batches to be logged in the optimizer, in which the batches " +
"are specified by their batch names and separated by comma.")
.version("3.0.0")
.stringConf
.createOptional
val DYNAMIC_PARTITION_PRUNING_ENABLED =
buildConf("spark.sql.optimizer.dynamicPartitionPruning.enabled")
.doc("When true, we will generate predicate for partition column when it's used as join key")
.version("3.0.0")
.booleanConf
.createWithDefault(true)
@ -247,6 +256,7 @@ object SQLConf {
.doc("When true, distinct count statistics will be used for computing the data size of the " +
"partitioned table after dynamic partition pruning, in order to evaluate if it is worth " +
"adding an extra subquery as the pruning filter if broadcast reuse is not applicable.")
.version("3.0.0")
.booleanConf
.createWithDefault(true)
@ -257,6 +267,7 @@ object SQLConf {
"used as the fallback filter ratio for computing the data size of the partitioned table " +
"after dynamic partition pruning, in order to evaluate if it is worth adding an extra " +
"subquery as the pruning filter if broadcast reuse is not applicable.")
.version("3.0.0")
.doubleConf
.createWithDefault(0.5)
@ -265,18 +276,21 @@ object SQLConf {
.internal()
.doc("When true, dynamic partition pruning will only apply when the broadcast exchange of " +
"a broadcast hash join operation can be reused as the dynamic pruning filter.")
.version("3.0.0")
.booleanConf
.createWithDefault(true)
val COMPRESS_CACHED = buildConf("spark.sql.inMemoryColumnarStorage.compressed")
.doc("When set to true Spark SQL will automatically select a compression codec for each " +
"column based on statistics of the data.")
.version("1.0.1")
.booleanConf
.createWithDefault(true)
val COLUMN_BATCH_SIZE = buildConf("spark.sql.inMemoryColumnarStorage.batchSize")
.doc("Controls the size of batches for columnar caching. Larger batch sizes can improve " +
"memory utilization and compression, but risk OOMs when caching data.")
.version("1.1.1")
.intConf
.createWithDefault(10000)
@ -284,6 +298,7 @@ object SQLConf {
buildConf("spark.sql.inMemoryColumnarStorage.partitionPruning")
.internal()
.doc("When true, enable partition pruning for in-memory columnar tables.")
.version("1.2.0")
.booleanConf
.createWithDefault(true)
@ -291,12 +306,14 @@ object SQLConf {
buildConf("spark.sql.inMemoryTableScanStatistics.enable")
.internal()
.doc("When true, enable in-memory table scan accumulators.")
.version("3.0.0")
.booleanConf
.createWithDefault(false)
val CACHE_VECTORIZED_READER_ENABLED =
buildConf("spark.sql.inMemoryColumnarStorage.enableVectorizedReader")
.doc("Enables vectorized reader for columnar caching.")
.version("2.3.1")
.booleanConf
.createWithDefault(true)
@ -304,12 +321,14 @@ object SQLConf {
buildConf("spark.sql.columnVector.offheap.enabled")
.internal()
.doc("When true, use OffHeapColumnVector in ColumnarBatch.")
.version("2.3.0")
.booleanConf
.createWithDefault(false)
val PREFER_SORTMERGEJOIN = buildConf("spark.sql.join.preferSortMergeJoin")
.internal()
.doc("When true, prefer sort merge join over shuffle hash join.")
.version("2.0.0")
.booleanConf
.createWithDefault(true)
@ -318,6 +337,7 @@ object SQLConf {
.doc("When true, enable use of radix sort when possible. Radix sort is much faster but " +
"requires additional memory to be reserved up-front. The memory overhead may be " +
"significant when sorting very small rows (up to 50% more in this case).")
.version("2.0.0")
.booleanConf
.createWithDefault(true)
@ -328,6 +348,7 @@ object SQLConf {
"command `ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan` has been " +
"run, and file-based data source tables where the statistics are computed directly on " +
"the files of data.")
.version("1.1.0")
.bytesConf(ByteUnit.BYTE)
.createWithDefaultString("10MB")
@ -336,6 +357,7 @@ object SQLConf {
.doc("Minimal increase rate in number of partitions between attempts when executing a take " +
"on a query. Higher values lead to more partitions read. Lower values might lead to " +
"longer execution times as more jobs will be run")
.version("2.1.1")
.intConf
.createWithDefault(4)
@ -343,6 +365,7 @@ object SQLConf {
buildConf("spark.sql.hive.advancedPartitionPredicatePushdown.enabled")
.internal()
.doc("When true, advanced partition predicate pushdown into Hive metastore is enabled.")
.version("2.3.0")
.booleanConf
.createWithDefault(true)
@ -350,12 +373,14 @@ object SQLConf {
.doc("The default number of partitions to use when shuffling data for joins or aggregations. " +
"Note: For structured streaming, this configuration cannot be changed between query " +
"restarts from the same checkpoint location.")
.version("1.1.0")
.intConf
.checkValue(_ > 0, "The value of spark.sql.shuffle.partitions must be positive")
.createWithDefault(200)
val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled")
.doc("When true, enable adaptive query execution.")
.version("1.6.0")
.booleanConf
.createWithDefault(false)
@ -365,6 +390,7 @@ object SQLConf {
"sub-queries. By setting this config to true (together with " +
s"'${ADAPTIVE_EXECUTION_ENABLED.key}' enabled), Spark will force apply adaptive query " +
"execution for all supported queries.")
.version("3.0.0")
.booleanConf
.createWithDefault(false)
@ -372,6 +398,7 @@ object SQLConf {
buildConf("spark.sql.adaptive.shuffle.reducePostShufflePartitions")
.doc(s"When true and '${ADAPTIVE_EXECUTION_ENABLED.key}' is enabled, this enables reducing " +
"the number of post-shuffle partitions based on map output statistics.")
.version("3.0.0")
.booleanConf
.createWithDefault(true)
@ -384,6 +411,7 @@ object SQLConf {
s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled, this feature also depends " +
"on a relocatable serializer, the concatenation support codec in use and the new version " +
"shuffle fetch protocol.")
.version("3.0.0")
.booleanConf
.createWithDefault(true)
@ -392,6 +420,7 @@ object SQLConf {
.doc("The advisory minimum number of post-shuffle partitions used when " +
s"'${ADAPTIVE_EXECUTION_ENABLED.key}' and " +
s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled.")
.version("3.0.0")
.intConf
.checkValue(_ > 0, "The minimum shuffle partition number " +
"must be a positive integer.")
@ -402,6 +431,7 @@ object SQLConf {
.doc("The target post-shuffle input size in bytes of a task. This configuration only has " +
s"an effect when '${ADAPTIVE_EXECUTION_ENABLED.key}' and " +
s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled.")
.version("1.6.0")
.bytesConf(ByteUnit.BYTE)
.createWithDefaultString("64MB")
@ -412,6 +442,7 @@ object SQLConf {
"spark.sql.shuffle.partitions. This configuration only has an effect when " +
s"'${ADAPTIVE_EXECUTION_ENABLED.key}' and " +
s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled.")
.version("3.0.0")
.intConf
.checkValue(_ > 0, "The maximum shuffle partition number " +
"must be a positive integer.")
@ -422,6 +453,7 @@ object SQLConf {
.doc(s"When true and '${ADAPTIVE_EXECUTION_ENABLED.key}' is enabled, this enables the " +
"optimization of converting the shuffle reader to local shuffle reader for the shuffle " +
"exchange of the broadcast hash join in probe side.")
.version("3.0.0")
.booleanConf
.createWithDefault(true)
@ -429,6 +461,7 @@ object SQLConf {
buildConf("spark.sql.adaptive.skewedJoinOptimization.enabled")
.doc("When true and adaptive execution is enabled, a skewed join is automatically handled at " +
"runtime.")
.version("3.0.0")
.booleanConf
.createWithDefault(true)
@ -437,6 +470,7 @@ object SQLConf {
.doc("A partition is considered as a skewed partition if its size is larger than" +
" this factor multiple the median partition size and also larger than " +
s" ${SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key}")
.version("3.0.0")
.intConf
.checkValue(_ > 0, "The skew factor must be positive.")
.createWithDefault(10)
@ -447,6 +481,7 @@ object SQLConf {
"considered as the build side of a broadcast-hash join in adaptive execution regardless " +
"of its size.This configuration only has an effect when " +
s"'${ADAPTIVE_EXECUTION_ENABLED.key}' is enabled.")
.version("3.0.0")
.doubleConf
.checkValue(_ >= 0, "The non-empty partition ratio must be positive number.")
.createWithDefault(0.2)
@ -455,6 +490,7 @@ object SQLConf {
buildConf("spark.sql.subexpressionElimination.enabled")
.internal()
.doc("When true, common subexpressions will be eliminated.")
.version("1.6.0")
.booleanConf
.createWithDefault(true)
@ -462,6 +498,7 @@ object SQLConf {
.internal()
.doc("Whether the query analyzer should be case sensitive or not. " +
"Default to case insensitive. It is highly discouraged to turn on case sensitive mode.")
.version("1.4.0")
.booleanConf
.createWithDefault(false)
@ -471,6 +508,7 @@ object SQLConf {
"plan to optimize them. Constraint propagation can sometimes be computationally expensive " +
"for certain kinds of query plans (such as those with a large number of predicates and " +
"aliases) which might negatively impact overall runtime.")
.version("2.2.0")
.booleanConf
.createWithDefault(true)
@ -479,6 +517,7 @@ object SQLConf {
.doc("When true, string literals (including regex patterns) remain escaped in our SQL " +
"parser. The default is false since Spark 2.0. Setting it to true can restore the behavior " +
"prior to Spark 2.0.")
.version("2.2.1")
.booleanConf
.createWithDefault(false)
@ -487,6 +526,7 @@ object SQLConf {
.doc("When estimating the output data size of a table scan, multiply the file size with this " +
"factor as the estimated data size, in case the data is compressed in the file and lead to" +
" a heavily underestimated result.")
.version("2.3.1")
.doubleConf
.checkValue(_ > 0, "the value of fileDataSizeFactor must be greater than 0")
.createWithDefault(1.0)
@ -495,6 +535,7 @@ object SQLConf {
.doc("When true, the Parquet data source merges schemas collected from all data files, " +
"otherwise the schema is picked from the summary file or a random data file " +
"if no summary file is available.")
.version("1.5.0")
.booleanConf
.createWithDefault(false)
@ -503,6 +544,7 @@ object SQLConf {
"summary files and we will ignore them when merging schema. Otherwise, if this is " +
"false, which is the default, we will merge all part-files. This should be considered " +
"as expert-only option, and shouldn't be enabled before knowing what it means exactly.")
.version("1.5.0")
.booleanConf
.createWithDefault(false)
@ -511,6 +553,7 @@ object SQLConf {
"Spark SQL, do not differentiate between binary data and strings when writing out the " +
"Parquet schema. This flag tells Spark SQL to interpret binary data as a string to provide " +
"compatibility with these systems.")
.version("1.1.1")
.booleanConf
.createWithDefault(false)
@ -519,6 +562,7 @@ object SQLConf {
"Spark would also store Timestamp as INT96 because we need to avoid precision lost of the " +
"nanoseconds field. This flag tells Spark SQL to interpret INT96 data as a timestamp to " +
"provide compatibility with these systems.")
.version("1.3.0")
.booleanConf
.createWithDefault(true)
@ -526,6 +570,7 @@ object SQLConf {
.doc("This controls whether timestamp adjustments should be applied to INT96 data when " +
"converting to timestamps, for data written by Impala. This is necessary because Impala " +
"stores INT96 data with a different timezone offset than Hive & Spark.")
.version("2.3.0")
.booleanConf
.createWithDefault(false)
@ -539,6 +584,7 @@ object SQLConf {
"is a standard timestamp type in Parquet, which stores number of microseconds from the " +
"Unix epoch. TIMESTAMP_MILLIS is also standard, but with millisecond precision, which " +
"means Spark has to truncate the microsecond portion of its timestamp value.")
.version("2.3.0")
.stringConf
.transform(_.toUpperCase(Locale.ROOT))
.checkValues(ParquetOutputTimestampType.values.map(_.toString))
@ -550,6 +596,7 @@ object SQLConf {
"precedence would be `compression`, `parquet.compression`, " +
"`spark.sql.parquet.compression.codec`. Acceptable values include: none, uncompressed, " +
"snappy, gzip, lzo, brotli, lz4, zstd.")
.version("1.1.1")
.stringConf
.transform(_.toLowerCase(Locale.ROOT))
.checkValues(Set("none", "uncompressed", "snappy", "gzip", "lzo", "lz4", "brotli", "zstd"))
@ -557,6 +604,7 @@ object SQLConf {
val PARQUET_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.parquet.filterPushdown")
.doc("Enables Parquet filter push-down optimization when set to true.")
.version("1.2.0")
.booleanConf
.createWithDefault(true)
@ -564,6 +612,7 @@ object SQLConf {
.doc("If true, enables Parquet filter push-down optimization for Date. " +
s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " +
"enabled.")
.version("2.4.0")
.internal()
.booleanConf
.createWithDefault(true)
@ -573,15 +622,17 @@ object SQLConf {
.doc("If true, enables Parquet filter push-down optimization for Timestamp. " +
s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " +
"enabled and Timestamp stored as TIMESTAMP_MICROS or TIMESTAMP_MILLIS type.")
.internal()
.booleanConf
.createWithDefault(true)
.version("2.4.0")
.internal()
.booleanConf
.createWithDefault(true)
val PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED =
buildConf("spark.sql.parquet.filterPushdown.decimal")
.doc("If true, enables Parquet filter push-down optimization for Decimal. " +
s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " +
"enabled.")
.version("2.4.0")
.internal()
.booleanConf
.createWithDefault(true)
@ -591,6 +642,7 @@ object SQLConf {
.doc("If true, enables Parquet filter push-down optimization for string startsWith function. " +
s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " +
"enabled.")
.version("2.4.0")
.internal()
.booleanConf
.createWithDefault(true)
@ -603,6 +655,7 @@ object SQLConf {
"By setting this value to 0 this feature can be disabled. " +
s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " +
"enabled.")
.version("2.4.0")
.internal()
.intConf
.checkValue(threshold => threshold >= 0, "The threshold must not be negative.")
@ -614,6 +667,7 @@ object SQLConf {
"systems such as Apache Hive and Apache Impala use. If false, the newer format in Parquet " +
"will be used. For example, decimals will be written in int-based format. If Parquet " +
"output is intended for use with systems that do not support this newer format, set to true.")
.version("1.6.0")
.booleanConf
.createWithDefault(false)
@ -623,6 +677,7 @@ object SQLConf {
"of org.apache.parquet.hadoop.ParquetOutputCommitter. If it is not, then metadata " +
"summaries will never be created, irrespective of the value of " +
"parquet.summary.metadata.level")
.version("1.5.0")
.internal()
.stringConf
.createWithDefault("org.apache.parquet.hadoop.ParquetOutputCommitter")
@ -630,6 +685,7 @@ object SQLConf {
val PARQUET_VECTORIZED_READER_ENABLED =
buildConf("spark.sql.parquet.enableVectorizedReader")
.doc("Enables vectorized parquet decoding.")
.version("2.0.0")
.booleanConf
.createWithDefault(true)
@ -639,12 +695,14 @@ object SQLConf {
s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' " +
"is enabled and the vectorized reader is not used. You can ensure the vectorized reader " +
s"is not used by setting '${PARQUET_VECTORIZED_READER_ENABLED.key}' to false.")
.version("2.3.0")
.booleanConf
.createWithDefault(false)
val PARQUET_VECTORIZED_READER_BATCH_SIZE = buildConf("spark.sql.parquet.columnarReaderBatchSize")
.doc("The number of rows to include in a parquet vectorized reader batch. The number should " +
"be carefully chosen to minimize overhead and avoid OOMs in reading data.")
.version("2.4.0")
.intConf
.createWithDefault(4096)