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

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 320ece106d38 [SPARK-55928][SQL] New linter for config effectiveness in 
views and UDFs
320ece106d38 is described below

commit 320ece106d38c1df7339eeb0763a6c3e0e5624f6
Author: Mihailo Timotic <[email protected]>
AuthorDate: Thu Mar 12 13:51:01 2026 +0800

    [SPARK-55928][SQL] New linter for config effectiveness in views and UDFs
    
    ### What changes were proposed in this pull request?
    This PR introduces a `ConfigBindingPolicy` framework that enforces all 
newly added Spark configurations to explicitly declare how their values are 
bound when used within SQL views or UDFs.
    #### Background: Conf + views mechanics
    There are 3 ways Spark configs can interact with views:
    1. The conf value is **stored with a view/UDF on creation** and is applied 
on read. Session value is deprioritized. Example: ANSI conf, timezone.
    2. The conf is **not stored with a view**, but its value is **propagated 
through a view from the active session**. Example: kill-switches, feature flags.
    3. The conf is **neither stored with a view, nor propagated** through a 
view. This is the **historical default** in Spark.
    The confusion arises for configurations that are not captured on view/UDF 
creation, but still need to be used when querying them. The common assumption 
is that if a conf is not preserved upon creation, its value inside the view/UDF 
will be whatever the value is in the currently active session. **This is not 
true.**
    If a conf is not preserved on creation, its value when querying the 
view/UDF will be:
    - The value from the currently active session, **only if** the conf is in a 
hardcoded allowlist (`RETAINED_ANALYSIS_FLAGS`).
    - The Spark default otherwise.
    This allowlist is extremely non-obvious and easy to forget about. This has 
caused regressions in the past where new configs affecting query semantics were 
not added to the allowlist, causing views and UDFs to silently use Spark 
defaults instead of session values.
    #### Changes
    1. **New `ConfigBindingPolicy` enum** (`common/utils`) with three values:
       - `SESSION`: The config value propagates from the active session to 
views/UDFs.
       - `PERSISTED`: The config uses the value saved at view/UDF creation 
time, or the Spark default if none was saved.
       - `NOT_APPLICABLE`: The config does not interact with view/UDF 
resolution. If accessed at runtime, it behaves the same as `SESSION`.
    2. **`ConfigBuilder.withBindingPolicy()`**: A new builder method to declare 
the binding policy when defining a config.
    3. **`ConfigEntry.bindingPolicy`**: A new field on all config entries to 
store the declared policy.
    4. **Dynamic retained config resolution in `Analyzer`**: Replaces the 
hardcoded `RETAINED_ANALYSIS_FLAGS` list with a dynamic lookup that retains all 
configs with `SESSION` or `NOT_APPLICABLE` binding policy when resolving views 
and SQL UDFs.
    5. **Binding policy annotations**: Added `withBindingPolicy(SESSION)` to 
configs that were previously in the hardcoded `RETAINED_ANALYSIS_FLAGS` list:
       - `PLAN_CHANGE_LOG_LEVEL`, `EXPRESSION_TREE_CHANGE_LOG_LEVEL`, 
`VIEW_SCHEMA_EVOLUTION_PRESERVE_USER_COMMENTS` (in `SQLConf`)
       - `CONVERT_METASTORE_PARQUET`, `CONVERT_METASTORE_ORC`, 
`CONVERT_INSERTING_PARTITIONED_TABLE`, `CONVERT_METASTORE_CTAS` (in `HiveUtils`)
    6. **Enforcement test** (`SparkConfigBindingPolicySuite`): A new test suite 
that fails if any newly added config does not declare a `bindingPolicy`, unless 
it is in an explicit exceptions allowlist. Existing configs without a binding 
policy have been grandfathered into the allowlist.
    #### When to use which policy
    - **`SESSION`** is the most common policy. Use it for feature flags or 
bugfix kill-switches where uniform behavior across the entire query is desired. 
Examples: enabling single-pass analyzer 
(`spark.sql.analyzer.singlePassResolver.enabledTentatively`), plan change 
logging (`spark.sql.planChangeLog.level`), bugfixes 
(`spark.sql.analyzer.preferColumnOverLcaInArrayIndex`). Think about it this 
way: if you make a behavior change and roll it out on by default, then discover 
a bug and need t [...]
    - **`PERSISTED`** should be used for configs that carry view semantic 
meaning that should be consistent regardless of session changes. A good example 
is ANSI mode -- views created with ANSI off should always have ANSI off, 
regardless of the session value. Examples: ANSI mode, session timezone.
    - **`NOT_APPLICABLE`** should be used for configs that don't interact with 
view/UDF resolution at all. Only choose this if you are confident the config 
doesn't interact with view/UDF analysis. Examples: UI confs, server confs.
    #### Why are all confs affected by the linter?
    Even within analysis, Spark can trigger a Spark job recursively which would 
potentially reference any conf (for example, this is needed for schema 
inference). The linter is active for all newly added confs regardless of 
whether they directly interact with view analysis.
    #### Why not fix all existing confs?
    Currently, there are over a thousand distinct configs in Spark. Fixing 
every single conf would introduce behavior changes. The linter only enforces 
the policy on new additions. Existing confs have been added to an exceptions 
allowlist. The long-term goal is to have all configs declare a binding policy 
and remove the exceptions allowlist entirely.
    ### Why are the changes needed?
    The `Analyzer.RETAINED_ANALYSIS_FLAGS` list was a manually maintained 
hardcoded allowlist of configs that should propagate from the active session 
when resolving views and SQL UDFs. This approach is error-prone: developers 
adding new configs that affect query semantics could easily forget to add them 
to this list, causing subtle bugs where views and UDFs silently use Spark 
defaults instead of session values.
    By requiring an explicit `ConfigBindingPolicy` declaration on every new 
config, developers are forced to think about how their config interacts with 
views and UDFs at definition time. The enforcement test catches any new config 
that lacks this declaration, preventing regressions.
    ### Does this PR introduce _any_ user-facing change?
    No.
    ### How was this patch tested?
    New test suite `SparkConfigBindingPolicySuite` with three tests:
    - `Test adding bindingPolicy to config`: Verifies a config with `SESSION` 
policy has the correct binding policy set.
    - `Config enforcement for bindingPolicy`: Ensures all registered configs 
either have a `bindingPolicy` declared or are in the exceptions allowlist, and 
that configs with `bindingPolicy` set are not redundantly in the allowlist.
    - `configs-without-binding-policy-exceptions file should be sorted 
alphabetically`: Validates the exceptions file ordering.
    ### Was this patch authored or co-authored using generative AI tooling?
    Generated-by: Cursor with Claude claude-4.6-opus-high-thinking
    
    Closes #54653 from mihailotim-db/mihailo-timotic_data/config_binding_policy.
    
    Authored-by: Mihailo Timotic <[email protected]>
    Signed-off-by: Wenchen Fan <[email protected]>
---
 .../internal/config/ConfigBindingPolicy.scala      |   54 +
 .../spark/internal/config/ConfigBuilder.scala      |   32 +-
 .../apache/spark/internal/config/ConfigEntry.scala |   35 +-
 dev/.rat-excludes                                  |    1 +
 .../spark/sql/catalyst/analysis/Analyzer.scala     |   31 +-
 .../org/apache/spark/sql/internal/SQLConf.scala    |    5 +-
 .../org/apache/spark/sql/hive/HiveUtils.scala      |    6 +
 .../configs-without-binding-policy-exceptions      | 1226 ++++++++++++++++++++
 .../SparkConfigBindingPolicySuite.scala            |   92 ++
 9 files changed, 1447 insertions(+), 35 deletions(-)

diff --git 
a/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigBindingPolicy.scala
 
b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigBindingPolicy.scala
new file mode 100644
index 000000000000..65009159f609
--- /dev/null
+++ 
b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigBindingPolicy.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.internal.config
+
+/**
+ * Defines how a configuration value is bound when used within SQL views, 
UDFs, or procedures.
+ *
+ * This enum controls whether a config value propagates from the active 
session or uses the value
+ * saved during view/UDF/procedure creation. If the policy is PERSISTED, but 
there is no saved
+ * value, a Spark default value is used.
+ *
+ * This is particularly important for configs that affect query behavior and 
where views/UDFs/
+ * procedures should change their behavior based on the caller's session 
settings. If the policy
+ * is PERSISTED, session-level config changes will not apply to 
views/UDFs/procedures but only
+ * to outer queries. In order for session-level changes to propagate 
correctly, this value must
+ * be explicitly set to SESSION.
+ */
+object ConfigBindingPolicy extends Enumeration {
+  type ConfigBindingPolicy = Value
+
+  /**
+   * The config value propagates from the active session to 
views/UDFs/procedures.
+   * This is important for queries that should have uniform behavior across 
the entire query.
+   */
+  val SESSION: Value = Value("SESSION")
+
+  /**
+   * The config uses the value saved on view/UDF/procedure creation if it 
exists,
+   * or Spark default value for that config if it doesn't.
+   */
+  val PERSISTED: Value = Value("PERSISTED")
+
+  /**
+   * The config does not apply to views/UDFs/procedures. If this config is 
accessed during
+   * view/UDF/procedure resolution, the value will be read from the active 
session (same as
+   * [[SESSION]]).
+   */
+  val NOT_APPLICABLE: Value = Value("NOT_APPLICABLE")
+}
diff --git 
a/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
 
b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
index c96324608ba5..a914a9638459 100644
--- 
a/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
+++ 
b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
@@ -211,7 +211,7 @@ private[spark] class TypedConfigBuilder[T](
   def createOptional: OptionalConfigEntry[T] = {
     val entry = new OptionalConfigEntry[T](parent.key, parent._prependedKey,
       parent._prependSeparator, parent._alternatives, converter, 
stringConverter, parent._doc,
-      parent._public, parent._version)
+      parent._public, parent._version, parent._bindingPolicy)
     parent._onCreate.foreach(_(entry))
     entry
   }
@@ -227,7 +227,8 @@ private[spark] class TypedConfigBuilder[T](
         val transformedDefault = converter(stringConverter(default))
         val entry = new ConfigEntryWithDefault[T](parent.key, 
parent._prependedKey,
           parent._prependSeparator, parent._alternatives, transformedDefault, 
converter,
-          stringConverter, parent._doc, parent._public, parent._version)
+          stringConverter, parent._doc, parent._public, parent._version,
+          parent._bindingPolicy)
         parent._onCreate.foreach(_ (entry))
         entry
     }
@@ -237,7 +238,7 @@ private[spark] class TypedConfigBuilder[T](
   def createWithDefaultFunction(defaultFunc: () => T): ConfigEntry[T] = {
     val entry = new ConfigEntryWithDefaultFunction[T](parent.key, 
parent._prependedKey,
       parent._prependSeparator, parent._alternatives, defaultFunc, converter, 
stringConverter,
-      parent._doc, parent._public, parent._version)
+      parent._doc, parent._public, parent._version, parent._bindingPolicy)
     parent._onCreate.foreach(_ (entry))
     entry
   }
@@ -249,7 +250,7 @@ private[spark] class TypedConfigBuilder[T](
   def createWithDefaultString(default: String): ConfigEntry[T] = {
     val entry = new ConfigEntryWithDefaultString[T](parent.key, 
parent._prependedKey,
       parent._prependSeparator, parent._alternatives, default, converter, 
stringConverter,
-      parent._doc, parent._public, parent._version)
+      parent._doc, parent._public, parent._version, parent._bindingPolicy)
     parent._onCreate.foreach(_(entry))
     entry
   }
@@ -272,6 +273,27 @@ private[spark] case class ConfigBuilder(key: String) {
   private[config] var _version = ""
   private[config] var _onCreate: Option[ConfigEntry[_] => Unit] = None
   private[config] var _alternatives = List.empty[String]
+  private[config] var _bindingPolicy: Option[ConfigBindingPolicy.Value] = None
+
+  /**
+   * Sets the binding policy for how this config value behaves within SQL 
views, UDFs, or
+   * procedures.
+   *
+   * - [[ConfigBindingPolicy.SESSION]]: The config value propagates from the 
active session
+   *   to views/UDFs/procedures. This is important for queries that should 
have uniform behavior
+   *   across the entire query.
+   *
+   * - [[ConfigBindingPolicy.PERSISTED]]: The view/UDF/procedure will use the 
value saved on
+   *   view/UDF/procedure creation if it exists, or Spark default value for 
that config if it
+   *   doesn't.
+   *
+   * - [[ConfigBindingPolicy.NOT_APPLICABLE]]: The config does not interact 
with view/UDF/procedure
+   *   resolution. If accessed at runtime, it behaves the same as 
[[ConfigBindingPolicy.SESSION]].
+   */
+  def withBindingPolicy(policy: ConfigBindingPolicy.Value): ConfigBuilder = {
+    _bindingPolicy = Some(policy)
+    this
+  }
 
   def internal(): ConfigBuilder = {
     _public = false
@@ -354,7 +376,7 @@ private[spark] case class ConfigBuilder(key: String) {
 
   def fallbackConf[T](fallback: ConfigEntry[T]): ConfigEntry[T] = {
     val entry = new FallbackConfigEntry(key, _prependedKey, _prependSeparator, 
_alternatives, _doc,
-      _public, _version, fallback)
+      _public, _version, _bindingPolicy, fallback)
     _onCreate.foreach(_(entry))
     entry
   }
diff --git 
a/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala
 
b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala
index 17d3329e6b49..10fcf3d076ee 100644
--- 
a/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala
+++ 
b/common/utils/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala
@@ -80,7 +80,8 @@ private[spark] abstract class ConfigEntry[T] (
     val stringConverter: T => String,
     val doc: String,
     val isPublic: Boolean,
-    val version: String) {
+    val version: String,
+    val bindingPolicy: Option[ConfigBindingPolicy.Value] = None) {
 
   import ConfigEntry._
 
@@ -106,7 +107,7 @@ private[spark] abstract class ConfigEntry[T] (
 
   override def toString: String = {
     s"ConfigEntry(key=$key, defaultValue=$defaultValueString, doc=$doc, " +
-      s"public=$isPublic, version=$version)"
+      s"public=$isPublic, version=$version, bindingPolicy=$bindingPolicy)"
   }
 }
 
@@ -120,7 +121,8 @@ private class ConfigEntryWithDefault[T] (
     stringConverter: T => String,
     doc: String,
     isPublic: Boolean,
-    version: String)
+    version: String,
+    bindingPolicy: Option[ConfigBindingPolicy.Value] = None)
   extends ConfigEntry(
     key,
     prependedKey,
@@ -130,7 +132,8 @@ private class ConfigEntryWithDefault[T] (
     stringConverter,
     doc,
     isPublic,
-    version
+    version,
+    bindingPolicy
   ) {
 
   override def defaultValue: Option[T] = Some(_defaultValue)
@@ -152,7 +155,8 @@ private class ConfigEntryWithDefaultFunction[T] (
     stringConverter: T => String,
     doc: String,
     isPublic: Boolean,
-    version: String)
+    version: String,
+    bindingPolicy: Option[ConfigBindingPolicy.Value] = None)
   extends ConfigEntry(
     key,
     prependedKey,
@@ -162,7 +166,8 @@ private class ConfigEntryWithDefaultFunction[T] (
     stringConverter,
     doc,
     isPublic,
-    version
+    version,
+    bindingPolicy
   ) {
 
   override def defaultValue: Option[T] = Some(_defaultFunction())
@@ -184,7 +189,8 @@ private class ConfigEntryWithDefaultString[T] (
     stringConverter: T => String,
     doc: String,
     isPublic: Boolean,
-    version: String)
+    version: String,
+    bindingPolicy: Option[ConfigBindingPolicy.Value] = None)
   extends ConfigEntry(
     key,
     prependedKey,
@@ -194,7 +200,8 @@ private class ConfigEntryWithDefaultString[T] (
     stringConverter,
     doc,
     isPublic,
-    version
+    version,
+    bindingPolicy
   ) {
 
   override def defaultValue: Option[T] = Some(valueConverter(_defaultValue))
@@ -220,7 +227,8 @@ private[spark] class OptionalConfigEntry[T](
     val rawStringConverter: T => String,
     doc: String,
     isPublic: Boolean,
-    version: String)
+    version: String,
+    bindingPolicy: Option[ConfigBindingPolicy.Value] = None)
   extends ConfigEntry[Option[T]](
     key,
     prependedKey,
@@ -230,7 +238,8 @@ private[spark] class OptionalConfigEntry[T](
     v => v.map(rawStringConverter).orNull,
     doc,
     isPublic,
-    version
+    version,
+    bindingPolicy
   ) {
 
   override def defaultValueString: String = ConfigEntry.UNDEFINED
@@ -251,6 +260,7 @@ private[spark] class FallbackConfigEntry[T] (
     doc: String,
     isPublic: Boolean,
     version: String,
+    bindingPolicy: Option[ConfigBindingPolicy.Value] = None,
     val fallback: ConfigEntry[T])
   extends ConfigEntry[T](
     key,
@@ -261,7 +271,8 @@ private[spark] class FallbackConfigEntry[T] (
     fallback.stringConverter,
     doc,
     isPublic,
-    version
+    version,
+    bindingPolicy
   ) {
 
   override def defaultValueString: String = s"<value of ${fallback.key}>"
@@ -285,4 +296,6 @@ private[spark] object ConfigEntry {
 
   def findEntry(key: String): ConfigEntry[_] = knownConfigs.get(key)
 
+  def listAllEntries(): java.util.Collection[ConfigEntry[_]] = 
knownConfigs.values()
+
 }
diff --git a/dev/.rat-excludes b/dev/.rat-excludes
index 0fa4dcb3b71e..224e09003752 100644
--- a/dev/.rat-excludes
+++ b/dev/.rat-excludes
@@ -113,6 +113,7 @@ vote.tmpl
 GangliaReporter.java
 application_1578436911597_0052
 config.properties
+configs-without-binding-policy-exceptions
 local-1596020211915
 app-20200706201101-0003
 application_1628109047826_1317105
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index e6d1d6da06b6..3f385ae0b49a 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -26,6 +26,7 @@ import scala.jdk.CollectionConverters._
 import scala.util.{Failure, Random, Success, Try}
 
 import org.apache.spark.{SparkException, SparkThrowable, 
SparkUnsupportedOperationException}
+import org.apache.spark.internal.config.ConfigBindingPolicy
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst._
 import org.apache.spark.sql.catalyst.analysis.resolver.{
@@ -239,29 +240,23 @@ object AnalysisContext {
 }
 
 object Analyzer {
-  // List of configurations that should be passed on when resolving views and 
SQL UDF.
-  private val RETAINED_ANALYSIS_FLAGS = Seq(
-    "spark.sql.view.schemaEvolution.preserveUserComments",
-    // retainedHiveConfigs
-    // TODO: remove these Hive-related configs after the `RelationConversions` 
is moved to
-    // optimization phase.
-    "spark.sql.hive.convertMetastoreParquet",
-    "spark.sql.hive.convertMetastoreOrc",
-    "spark.sql.hive.convertInsertingPartitionedTable",
-    "spark.sql.hive.convertInsertingUnpartitionedTable",
-    "spark.sql.hive.convertMetastoreCtas",
-    // retainedLoggingConfigs
-    "spark.sql.planChangeLog.level",
-    "spark.sql.expressionTreeChangeLog.level"
-  )
-
+  // Configs with bindingPolicy SESSION or NOT_APPLICABLE are retained when 
resolving views and
+  // SQL UDFs, so that their values propagate from the active session rather 
than falling back to
+  // Spark defaults. Note: configs defined in lazily-loaded modules (e.g., 
sql/hive) will only
+  // be included if their holding Scala object has been initialized before 
this set is computed.
   def retainResolutionConfigsForAnalysis(
       newConf: SQLConf,
       existingConf: SQLConf,
       createSparkVersion: String = ""): Unit = {
+    val retainedConfigKeys = SQLConf.getConfigEntries().asScala
+      .filter(entry =>
+        entry.bindingPolicy.contains(ConfigBindingPolicy.SESSION) ||
+        entry.bindingPolicy.contains(ConfigBindingPolicy.NOT_APPLICABLE))
+      .map(_.key)
+      .toSet
+
     val retainedConfigs = existingConf.getAllConfs.filter { case (key, _) =>
-      // Also apply catalog configs
-      RETAINED_ANALYSIS_FLAGS.contains(key) || 
key.startsWith("spark.sql.catalog.")
+      retainedConfigKeys.contains(key) || key.startsWith("spark.sql.catalog.")
     }
 
     retainedConfigs.foreach { case (k, v) =>
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 84f2f6b90aa7..c90cf6388684 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -88,7 +88,7 @@ object SQLConf {
     sqlConfEntries.get(key)
   }
 
-  private[internal] def getConfigEntries(): util.Collection[ConfigEntry[_]] = {
+  private[sql] def getConfigEntries(): util.Collection[ConfigEntry[_]] = {
     sqlConfEntries.values()
   }
 
@@ -544,6 +544,7 @@ object SQLConf {
       s"plan after a rule or batch is applied. The value can be " +
       s"${VALID_LOG_LEVELS.mkString(", ")}.")
     .version("3.1.0")
+    .withBindingPolicy(ConfigBindingPolicy.SESSION)
     .enumConf(classOf[Level])
     .createWithDefault(Level.TRACE)
 
@@ -577,6 +578,7 @@ object SQLConf {
       "the resolved expression tree in the single-pass bottom-up Resolver. The 
value can be " +
       s"${VALID_LOG_LEVELS.mkString(", ")}.")
     .version("4.0.0")
+    .withBindingPolicy(ConfigBindingPolicy.SESSION)
     .enumConf(classOf[Level])
     .createWithDefault(Level.TRACE)
 
@@ -2258,6 +2260,7 @@ object SQLConf {
         "when the underlying table schema evolves. When disabled, view 
comments will be " +
         "overwritten with table comments on every schema sync.")
       .version("4.2.0")
+      .withBindingPolicy(ConfigBindingPolicy.SESSION)
       .booleanConf
       .createWithDefault(true)
 
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
index f0e1e208e542..4028da153ff9 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
@@ -38,6 +38,7 @@ import org.apache.spark.SparkConf
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.LogKeys
+import org.apache.spark.internal.config.ConfigBindingPolicy
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.classic.SQLContext
 import org.apache.spark.sql.execution.command.DDLUtils
@@ -126,6 +127,7 @@ private[spark] object HiveUtils extends Logging {
     .doc("When set to true, the built-in Parquet reader and writer are used to 
process " +
       "parquet tables created by using the HiveQL syntax, instead of Hive 
serde.")
     .version("1.1.1")
+    .withBindingPolicy(ConfigBindingPolicy.SESSION)
     .booleanConf
     .createWithDefault(true)
 
@@ -142,6 +144,7 @@ private[spark] object HiveUtils extends Logging {
     .doc("When set to true, the built-in ORC reader and writer are used to 
process " +
       "ORC tables created by using the HiveQL syntax, instead of Hive serde.")
     .version("2.0.0")
+    .withBindingPolicy(ConfigBindingPolicy.SESSION)
     .booleanConf
     .createWithDefault(true)
 
@@ -152,6 +155,7 @@ private[spark] object HiveUtils extends Logging {
         "to process inserting into partitioned ORC/Parquet tables created by 
using the HiveSQL " +
         "syntax.")
       .version("3.0.0")
+      .withBindingPolicy(ConfigBindingPolicy.SESSION)
       .booleanConf
       .createWithDefault(true)
 
@@ -162,6 +166,7 @@ private[spark] object HiveUtils extends Logging {
         "to process inserting into unpartitioned ORC/Parquet tables created by 
using the HiveSQL " +
         "syntax.")
       .version("4.0.0")
+      .withBindingPolicy(ConfigBindingPolicy.SESSION)
       .booleanConf
       .createWithDefault(true)
 
@@ -171,6 +176,7 @@ private[spark] object HiveUtils extends Logging {
       "`spark.sql.hive.convertMetastoreParquet` or 
`spark.sql.hive.convertMetastoreOrc` is " +
       "enabled respectively for Parquet and ORC formats")
     .version("3.0.0")
+    .withBindingPolicy(ConfigBindingPolicy.SESSION)
     .booleanConf
     .createWithDefault(true)
 
diff --git 
a/sql/hive/src/test/resources/conf/binding-policy-exceptions/configs-without-binding-policy-exceptions
 
b/sql/hive/src/test/resources/conf/binding-policy-exceptions/configs-without-binding-policy-exceptions
new file mode 100644
index 000000000000..f20b592daf5a
--- /dev/null
+++ 
b/sql/hive/src/test/resources/conf/binding-policy-exceptions/configs-without-binding-policy-exceptions
@@ -0,0 +1,1226 @@
+spark.acls.enable
+spark.admin.acls
+spark.admin.acls.groups
+spark.api.mode
+spark.app.attempt.id
+spark.appStateStore.asyncTracking.enable
+spark.archives
+spark.authenticate
+spark.authenticate.enableSaslEncryption
+spark.authenticate.secret
+spark.authenticate.secret.driver.file
+spark.authenticate.secret.executor.file
+spark.authenticate.secret.file
+spark.authenticate.secretBitLength
+spark.barrier.sync.timeout
+spark.block.failures.beforeLocationRefresh
+spark.blockManager.port
+spark.broadcast.UDFCompressionThreshold
+spark.broadcast.blockSize
+spark.broadcast.checksum
+spark.broadcast.compress
+spark.buffer.pageSize
+spark.buffer.size
+spark.buffer.write.chunkSize
+spark.checkpoint.compress
+spark.checkpoint.dir
+spark.cleaner.periodicGC.interval
+spark.cleaner.referenceTracking
+spark.cleaner.referenceTracking.blocking
+spark.cleaner.referenceTracking.blocking.shuffle
+spark.cleaner.referenceTracking.blocking.timeout
+spark.cleaner.referenceTracking.cleanCheckpoints
+spark.connect.scalaUdf.stubPrefixes
+spark.cores.max
+spark.decommission.enabled
+spark.default.parallelism
+spark.diskStore.subDirectories
+spark.driver.bindAddress
+spark.driver.blockManager.port
+spark.driver.cores
+spark.driver.defaultExtraClassPath
+spark.driver.extraClassPath
+spark.driver.extraJavaOptions
+spark.driver.extraLibraryPath
+spark.driver.host
+spark.driver.log.allowErasureCoding
+spark.driver.log.dfsDir
+spark.driver.log.layout
+spark.driver.log.localDir
+spark.driver.log.persistToDfs.enabled
+spark.driver.log.redirectConsoleOutputs
+spark.driver.maxResultSize
+spark.driver.memory
+spark.driver.memoryOverhead
+spark.driver.memoryOverheadFactor
+spark.driver.metrics.pollingInterval
+spark.driver.minMemoryOverhead
+spark.driver.port
+spark.driver.resourcesFile
+spark.driver.supervise
+spark.driver.timeout
+spark.driver.userClassPathFirst
+spark.dynamicAllocation.cachedExecutorIdleTimeout
+spark.dynamicAllocation.enabled
+spark.dynamicAllocation.executorAllocationRatio
+spark.dynamicAllocation.executorIdleTimeout
+spark.dynamicAllocation.initialExecutors
+spark.dynamicAllocation.maxExecutors
+spark.dynamicAllocation.minExecutors
+spark.dynamicAllocation.schedulerBacklogTimeout
+spark.dynamicAllocation.shuffleTracking.enabled
+spark.dynamicAllocation.shuffleTracking.timeout
+spark.dynamicAllocation.sustainedSchedulerBacklogTimeout
+spark.dynamicAllocation.testing
+spark.eventLog.buffer.kb
+spark.eventLog.compress
+spark.eventLog.compression.codec
+spark.eventLog.dir
+spark.eventLog.enabled
+spark.eventLog.erasureCoding.enabled
+spark.eventLog.excludedPatterns
+spark.eventLog.gcMetrics.oldGenerationGarbageCollectors
+spark.eventLog.gcMetrics.youngGenerationGarbageCollectors
+spark.eventLog.includeTaskMetricsAccumulators
+spark.eventLog.logBlockUpdates.enabled
+spark.eventLog.logStageExecutorMetrics
+spark.eventLog.longForm.enabled
+spark.eventLog.overwrite
+spark.eventLog.rolling.enabled
+spark.eventLog.rolling.maxFileSize
+spark.eventLog.testing
+spark.excludeOnFailure.application.enabled
+spark.excludeOnFailure.application.fetchFailure.enabled
+spark.excludeOnFailure.application.maxFailedExecutorsPerNode
+spark.excludeOnFailure.application.maxFailedTasksPerExecutor
+spark.excludeOnFailure.enabled
+spark.excludeOnFailure.killExcludedExecutors
+spark.excludeOnFailure.killExcludedExecutors.decommission
+spark.excludeOnFailure.stage.maxFailedExecutorsPerNode
+spark.excludeOnFailure.stage.maxFailedTasksPerExecutor
+spark.excludeOnFailure.task.maxTaskAttemptsPerExecutor
+spark.excludeOnFailure.task.maxTaskAttemptsPerNode
+spark.excludeOnFailure.taskAndStage.enabled
+spark.excludeOnFailure.timeout
+spark.executor.allowSparkContext
+spark.executor.cores
+spark.executor.decommission.forceKillTimeout
+spark.executor.decommission.killInterval
+spark.executor.decommission.signal
+spark.executor.defaultExtraClassPath
+spark.executor.extraClassPath
+spark.executor.extraJavaOptions
+spark.executor.extraLibraryPath
+spark.executor.failuresValidityInterval
+spark.executor.heartbeat.dropZeroAccumulatorUpdates
+spark.executor.heartbeat.maxFailures
+spark.executor.heartbeatInterval
+spark.executor.id
+spark.executor.instances
+spark.executor.isolatedSessionCache.size
+spark.executor.killOnFatalError.depth
+spark.executor.logs.redirectConsoleOutputs
+spark.executor.logs.rolling.enableCompression
+spark.executor.logs.rolling.maxRetainedFiles
+spark.executor.logs.rolling.maxSize
+spark.executor.logs.rolling.strategy
+spark.executor.logs.rolling.time.interval
+spark.executor.maxNumFailures
+spark.executor.memory
+spark.executor.memoryOverhead
+spark.executor.memoryOverheadFactor
+spark.executor.metrics.fileSystemSchemes
+spark.executor.metrics.pollingInterval
+spark.executor.minMemoryOverhead
+spark.executor.processTreeMetrics.enabled
+spark.executor.pyspark.memory
+spark.executor.python.worker.log.details
+spark.executor.syncLogLevel.enabled
+spark.executor.userClassPathFirst
+spark.extraListeners
+spark.file.transferTo
+spark.files
+spark.files.fetchFailure.unRegisterOutputOnHost
+spark.files.ignoreCorruptFiles
+spark.files.ignoreMissingFiles
+spark.files.maxPartitionBytes
+spark.files.openCostInBytes
+spark.hadoopRDD.ignoreEmptySplits
+spark.history.custom.executor.log.url
+spark.history.custom.executor.log.url.applyIncompleteApplication
+spark.history.fs.cleaner.enabled
+spark.history.fs.cleaner.interval
+spark.history.fs.cleaner.maxAge
+spark.history.fs.cleaner.maxNum
+spark.history.fs.driverlog.cleaner.enabled
+spark.history.fs.driverlog.cleaner.interval
+spark.history.fs.driverlog.cleaner.maxAge
+spark.history.fs.endEventReparseChunkSize
+spark.history.fs.eventLog.rolling.compaction.score.threshold
+spark.history.fs.eventLog.rolling.maxFilesToRetain
+spark.history.fs.eventLog.rolling.onDemandLoadEnabled
+spark.history.fs.inProgressOptimization.enabled
+spark.history.fs.logDirectory
+spark.history.fs.logDirectory.names
+spark.history.fs.numCompactThreads
+spark.history.fs.numReplayThreads
+spark.history.fs.safemodeCheck.interval
+spark.history.fs.update.batchSize
+spark.history.fs.update.interval
+spark.history.kerberos.enabled
+spark.history.kerberos.keytab
+spark.history.kerberos.principal
+spark.history.provider
+spark.history.retainedApplications
+spark.history.store.hybridStore.diskBackend
+spark.history.store.hybridStore.enabled
+spark.history.store.hybridStore.maxMemoryUsage
+spark.history.store.maxDiskUsage
+spark.history.store.path
+spark.history.store.serializer
+spark.history.ui.acls.enable
+spark.history.ui.admin.acls
+spark.history.ui.admin.acls.groups
+spark.history.ui.maxApplications
+spark.history.ui.port
+spark.history.ui.title
+spark.io.compression.codec
+spark.io.compression.lz4.blockSize
+spark.io.compression.lzf.parallel.enabled
+spark.io.compression.snappy.blockSize
+spark.io.compression.zstd.bufferPool.enabled
+spark.io.compression.zstd.bufferSize
+spark.io.compression.zstd.level
+spark.io.compression.zstd.strategy
+spark.io.compression.zstd.workers
+spark.io.crypto.cipher.transformation
+spark.io.encryption.enabled
+spark.io.encryption.keySizeBits
+spark.io.encryption.keygen.algorithm
+spark.io.warning.largeFileThreshold
+spark.jars
+spark.jars.excludes
+spark.jars.ivy
+spark.jars.ivySettings
+spark.jars.packages
+spark.jars.repositories
+spark.kerberos.access.hadoopFileSystems
+spark.kerberos.keytab
+spark.kerberos.principal
+spark.kerberos.relogin.period
+spark.kerberos.renewal.credentials
+spark.kryo.classesToRegister
+spark.kryo.pool
+spark.kryo.referenceTracking
+spark.kryo.registrationRequired
+spark.kryo.registrator
+spark.kryo.unsafe
+spark.kryoserializer.buffer
+spark.kryoserializer.buffer.max
+spark.kubernetes.jars.avoidDownloadSchemes
+spark.locality.wait
+spark.locality.wait.legacyResetOnTaskLaunch
+spark.locality.wait.node
+spark.locality.wait.process
+spark.locality.wait.rack
+spark.log.callerContext
+spark.log.legacyTaskNameMdc.enabled
+spark.log.level
+spark.log.structuredLogging.enabled
+spark.master.rest.enabled
+spark.master.rest.filters
+spark.master.rest.host
+spark.master.rest.maxThreads
+spark.master.rest.port
+spark.master.rest.virtualThread.enabled
+spark.master.ui.decommission.allow.mode
+spark.master.ui.historyServerUrl
+spark.master.ui.port
+spark.master.ui.title
+spark.master.ui.visibleEnvVarPrefixes
+spark.master.useAppNameAsAppId.enabled
+spark.master.useDriverIdAsAppName.enabled
+spark.memory.fraction
+spark.memory.offHeap.enabled
+spark.memory.offHeap.size
+spark.memory.storageFraction
+spark.memory.unmanagedMemoryPollingInterval
+spark.metrics.appStatusSource.enabled
+spark.metrics.conf
+spark.metrics.executorMetricsSource.enabled
+spark.metrics.namespace
+spark.metrics.staticSources.enabled
+spark.modify.acls
+spark.modify.acls.groups
+spark.network.crypto.enabled
+spark.network.crypto.saslFallback
+spark.network.maxRemoteBlockSizeFetchToMem
+spark.network.remoteReadNioBufferConversion
+spark.network.timeout
+spark.network.timeoutInterval
+spark.plugins
+spark.pyspark.driver.python
+spark.pyspark.python
+spark.python.authenticate.socketTimeout
+spark.python.daemon.killWorkerOnFlushFailure
+spark.python.daemon.module
+spark.python.factory.idleWorkerMaxPoolSize
+spark.python.sql.dataFrameDebugging.enabled
+spark.python.task.killTimeout
+spark.python.unix.domain.socket.dir
+spark.python.unix.domain.socket.enabled
+spark.python.use.daemon
+spark.python.worker.faulthandler.enabled
+spark.python.worker.idleTimeoutSeconds
+spark.python.worker.killOnIdleTimeout
+spark.python.worker.module
+spark.python.worker.reuse
+spark.python.worker.tracebackDumpIntervalSeconds
+spark.rdd.cache.visibilityTracking.enabled
+spark.rdd.checkpoint.cachePreferredLocsExpireTime
+spark.rdd.compress
+spark.rdd.limit.initialNumPartitions
+spark.rdd.limit.scaleUpFactor
+spark.rdd.parallelListingThreshold
+spark.redaction.regex
+spark.redaction.string.regex
+spark.reducer.maxBlocksInFlightPerAddress
+spark.reducer.maxReqsInFlight
+spark.reducer.maxSizeInFlight
+spark.resources.discoveryPlugin
+spark.resources.warnings.testing
+spark.rpc.askTimeout
+spark.rpc.connect.threads
+spark.rpc.io.numConnectionsPerPeer
+spark.rpc.io.threads
+spark.rpc.lookupTimeout
+spark.rpc.message.maxSize
+spark.rpc.netty.dispatcher.numThreads
+spark.scheduler.allocation.file
+spark.scheduler.barrier.maxConcurrentTasksCheck.interval
+spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures
+spark.scheduler.dropTaskInfoAccumulablesOnTaskCompletion.enabled
+spark.scheduler.excludeOnFailure.unschedulableTaskSetTimeout
+spark.scheduler.executorTaskExcludeOnFailureTime
+spark.scheduler.listenerbus.eventqueue.capacity
+spark.scheduler.listenerbus.exitTimeout
+spark.scheduler.listenerbus.logSlowEvent
+spark.scheduler.listenerbus.logSlowEvent.threshold
+spark.scheduler.listenerbus.metrics.maxListenerClassesTimed
+spark.scheduler.maxRegisteredResourcesWaitingTime
+spark.scheduler.maxRetainedRemovedDecommissionExecutors
+spark.scheduler.maxRetainedUnknownDecommissionExecutors
+spark.scheduler.minRegisteredResourcesRatio
+spark.scheduler.mode
+spark.scheduler.numCancelledJobGroupsToTrack
+spark.scheduler.resource.profileMergeConflicts
+spark.scheduler.revive.interval
+spark.scheduler.stage.legacyAbortAfterKillTasks
+spark.security.credentials.renewalRatio
+spark.security.credentials.retryWait
+spark.serializer
+spark.serializer.extraDebugInfo
+spark.serializer.objectStreamReset
+spark.shuffle.accurateBlockSkewedFactor
+spark.shuffle.accurateBlockThreshold
+spark.shuffle.checksum.algorithm
+spark.shuffle.checksum.enabled
+spark.shuffle.compress
+spark.shuffle.detectCorrupt
+spark.shuffle.detectCorrupt.useExtraMemory
+spark.shuffle.file.buffer
+spark.shuffle.file.merge.buffer
+spark.shuffle.localDisk.file.output.buffer
+spark.shuffle.manager
+spark.shuffle.mapOutput.dispatcher.numThreads
+spark.shuffle.mapOutput.minSizeForBroadcast
+spark.shuffle.mapOutput.parallelAggregationThreshold
+spark.shuffle.mapStatus.compression.codec
+spark.shuffle.maxAccurateSkewedBlockNumber
+spark.shuffle.maxAttemptsOnNettyOOM
+spark.shuffle.minNumPartitionsToHighlyCompress
+spark.shuffle.push.enabled
+spark.shuffle.push.finalize.timeout
+spark.shuffle.push.maxBlockBatchSize
+spark.shuffle.push.maxBlockSizeToPush
+spark.shuffle.push.maxRetainedMergerLocations
+spark.shuffle.push.merge.finalizeThreads
+spark.shuffle.push.mergersMinStaticThreshold
+spark.shuffle.push.mergersMinThresholdRatio
+spark.shuffle.push.minCompletedPushRatio
+spark.shuffle.push.minShuffleSizeToWait
+spark.shuffle.push.numPushThreads
+spark.shuffle.push.results.timeout
+spark.shuffle.push.sendFinalizeRPCThreads
+spark.shuffle.readHostLocalDisk
+spark.shuffle.reduceLocality.enabled
+spark.shuffle.registration.maxAttempts
+spark.shuffle.registration.timeout
+spark.shuffle.service.db.backend
+spark.shuffle.service.db.enabled
+spark.shuffle.service.enabled
+spark.shuffle.service.fetch.rdd.enabled
+spark.shuffle.service.name
+spark.shuffle.service.port
+spark.shuffle.service.removeShuffle
+spark.shuffle.sort.bypassMergeThreshold
+spark.shuffle.sort.initialBufferSize
+spark.shuffle.sort.io.plugin.class
+spark.shuffle.sort.useRadixSort
+spark.shuffle.spill.batchSize
+spark.shuffle.spill.compress
+spark.shuffle.spill.diskWriteBufferSize
+spark.shuffle.spill.initialMemoryThreshold
+spark.shuffle.spill.maxSizeInBytesForSpillThreshold
+spark.shuffle.spill.numElementsForceSpillThreshold
+spark.shuffle.sync
+spark.shuffle.unsafe.fastMergeEnabled
+spark.shuffle.unsafe.file.output.buffer
+spark.shuffle.useOldFetchProtocol
+spark.shutdown.timeout
+spark.speculation
+spark.speculation.efficiency.enabled
+spark.speculation.efficiency.longRunTaskFactor
+spark.speculation.efficiency.processRateMultiplier
+spark.speculation.interval
+spark.speculation.minTaskRuntime
+spark.speculation.multiplier
+spark.speculation.quantile
+spark.speculation.task.duration.threshold
+spark.sql.SQLConfEntrySuite.boolean
+spark.sql.SQLConfEntrySuite.double
+spark.sql.SQLConfEntrySuite.duplicate
+spark.sql.SQLConfEntrySuite.enum
+spark.sql.SQLConfEntrySuite.int
+spark.sql.SQLConfEntrySuite.long
+spark.sql.SQLConfEntrySuite.optional
+spark.sql.SQLConfEntrySuite.string
+spark.sql.SQLConfEntrySuite.stringSeq
+spark.sql.adaptive.advisoryPartitionSizeInBytes
+spark.sql.adaptive.applyFinalStageShuffleOptimizations
+spark.sql.adaptive.autoBroadcastJoinThreshold
+spark.sql.adaptive.coalescePartitions.enabled
+spark.sql.adaptive.coalescePartitions.initialPartitionNum
+spark.sql.adaptive.coalescePartitions.minPartitionNum
+spark.sql.adaptive.coalescePartitions.minPartitionSize
+spark.sql.adaptive.coalescePartitions.parallelismFirst
+spark.sql.adaptive.customCostEvaluatorClass
+spark.sql.adaptive.enabled
+spark.sql.adaptive.fetchShuffleBlocksInBatch
+spark.sql.adaptive.forceApply
+spark.sql.adaptive.forceOptimizeSkewedJoin
+spark.sql.adaptive.localShuffleReader.enabled
+spark.sql.adaptive.logLevel
+spark.sql.adaptive.maxShuffledHashJoinLocalMapThreshold
+spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin
+spark.sql.adaptive.optimizeSkewsInRebalancePartitions.enabled
+spark.sql.adaptive.optimizer.excludedRules
+spark.sql.adaptive.rebalancePartitionsSmallPartitionFactor
+spark.sql.adaptive.shuffle.targetPostShuffleInputSize
+spark.sql.adaptive.skewJoin.enabled
+spark.sql.adaptive.skewJoin.skewedPartitionFactor
+spark.sql.adaptive.skewJoin.skewedPartitionThresholdInBytes
+spark.sql.adaptive.streaming.stateless.enabled
+spark.sql.addPartitionInBatch.size
+spark.sql.allowNamedFunctionArguments
+spark.sql.alwaysInlineCommonExpr
+spark.sql.analyzer.allowSubqueryExpressionsInLambdasOrHigherOrderFunctions
+spark.sql.analyzer.canonicalization.multiCommutativeOpMemoryOptThreshold
+spark.sql.analyzer.dontDeduplicateExpressionIfExprIdInOutput
+spark.sql.analyzer.expandTagPassthroughDuplicates
+spark.sql.analyzer.failAmbiguousSelfJoin
+spark.sql.analyzer.maxIterations
+spark.sql.analyzer.preferColumnOverLcaInArrayIndex
+spark.sql.analyzer.scalarSubqueryAllowGroupByColumnEqualToConstant
+spark.sql.analyzer.singlePassResolver.dualRunSampleRate
+spark.sql.analyzer.singlePassResolver.dualRunWithLegacy
+spark.sql.analyzer.singlePassResolver.enabled
+spark.sql.analyzer.singlePassResolver.enabledTentatively
+spark.sql.analyzer.singlePassResolver.exposeResolverGuardFailure
+spark.sql.analyzer.singlePassResolver.preventUsingAliasesFromNonDirectChildren
+spark.sql.analyzer.singlePassResolver.relationBridging.enabled
+spark.sql.analyzer.singlePassResolver.returnSinglePassResultInDualRun
+spark.sql.analyzer.singlePassResolver.runExtendedResolutionChecks
+spark.sql.analyzer.singlePassResolver.runHeavyExtendedResolutionChecks
+spark.sql.analyzer.singlePassResolver.throwFromResolverGuard
+spark.sql.analyzer.singlePassResolver.validationEnabled
+spark.sql.analyzer.sqlFunctionResolution.applyConfOverrides
+spark.sql.analyzer.subqueryAliasAlwaysPropagateMetadataColumns
+spark.sql.analyzer.unionIsResolvedWhenDuplicatesPerChildResolved
+spark.sql.analyzer.uniqueNecessaryMetadataColumns
+spark.sql.ansi.doubleQuotedIdentifiers
+spark.sql.ansi.enabled
+spark.sql.ansi.enforceReservedKeywords
+spark.sql.ansi.relationPrecedence
+spark.sql.artifact.cacheStorageLevel
+spark.sql.artifact.copyFromLocalToFs.allowDestLocal
+spark.sql.artifact.isolation.alwaysApplyClassloader
+spark.sql.artifact.isolation.enabled
+spark.sql.assumeAnsiFalseIfNotPersisted.enabled
+spark.sql.autoBroadcastJoinThreshold
+spark.sql.avro.compression.codec
+spark.sql.avro.datetimeRebaseModeInRead
+spark.sql.avro.datetimeRebaseModeInWrite
+spark.sql.avro.deflate.level
+spark.sql.avro.filterPushdown.enabled
+spark.sql.avro.xz.level
+spark.sql.avro.zstandard.bufferPool.enabled
+spark.sql.avro.zstandard.level
+spark.sql.binaryOutputStyle
+spark.sql.broadcastExchange.maxThreadThreshold
+spark.sql.broadcastTimeout
+spark.sql.bucketing.coalesceBucketsInJoin.enabled
+spark.sql.bucketing.coalesceBucketsInJoin.maxBucketRatio
+spark.sql.cache.serializer
+spark.sql.cartesianProductExec.buffer.in.memory.threshold
+spark.sql.cartesianProductExec.buffer.spill.size.threshold
+spark.sql.cartesianProductExec.buffer.spill.threshold
+spark.sql.caseSensitive
+spark.sql.catalog.spark_catalog
+spark.sql.catalog.spark_catalog.defaultDatabase
+spark.sql.catalogImplementation
+spark.sql.cbo.enabled
+spark.sql.cbo.joinReorder.card.weight
+spark.sql.cbo.joinReorder.dp.star.filter
+spark.sql.cbo.joinReorder.dp.threshold
+spark.sql.cbo.joinReorder.enabled
+spark.sql.cbo.planStats.enabled
+spark.sql.cbo.starJoinFTRatio
+spark.sql.cbo.starSchemaDetection
+spark.sql.charAsVarchar
+spark.sql.chunkBase64String.enabled
+spark.sql.classic.shuffleDependency.fileCleanup.enabled
+spark.sql.cli.print.header
+spark.sql.codegen.aggregate.fastHashMap.capacityBit
+spark.sql.codegen.aggregate.map.twolevel.enabled
+spark.sql.codegen.aggregate.map.twolevel.partialOnly
+spark.sql.codegen.aggregate.map.vectorized.enable
+spark.sql.codegen.aggregate.sortAggregate.enabled
+spark.sql.codegen.aggregate.splitAggregateFunc.enabled
+spark.sql.codegen.broadcastCleanedSourceThreshold
+spark.sql.codegen.cache.maxEntries
+spark.sql.codegen.comments
+spark.sql.codegen.factoryMode
+spark.sql.codegen.fallback
+spark.sql.codegen.hugeMethodLimit
+spark.sql.codegen.join.buildSideOuterShuffledHashJoin.enabled
+spark.sql.codegen.join.existenceSortMergeJoin.enabled
+spark.sql.codegen.join.fullOuterShuffledHashJoin.enabled
+spark.sql.codegen.join.fullOuterSortMergeJoin.enabled
+spark.sql.codegen.logLevel
+spark.sql.codegen.logging.maxLines
+spark.sql.codegen.maxFields
+spark.sql.codegen.methodSplitThreshold
+spark.sql.codegen.splitConsumeFuncByOperator
+spark.sql.codegen.useIdInClassName
+spark.sql.codegen.wholeStage
+spark.sql.collation.allowInMapKeys
+spark.sql.collation.objectLevel.enabled
+spark.sql.collation.schemaLevel.enabled
+spark.sql.collation.trim.enabled
+spark.sql.columnNameOfCorruptRecord
+spark.sql.columnVector.offheap.enabled
+spark.sql.connect.shuffleDependency.fileCleanup.enabled
+spark.sql.constraintPropagation.enabled
+spark.sql.crossJoin.enabled
+spark.sql.csv.filterPushdown.enabled
+spark.sql.csv.parser.columnPruning.enabled
+spark.sql.csv.parser.inputBufferSize
+spark.sql.cteRecursionAnchorRowsLimitToConvertToLocalRelation
+spark.sql.cteRecursionLevelLimit
+spark.sql.cteRecursionRowLimit
+spark.sql.cteRelationDefMaxRows.enabled
+spark.sql.dataFrameQueryContext.enabled
+spark.sql.dataSource.alwaysCreateV2Predicate
+spark.sql.dataSource.skipAssertOnPredicatePushdown
+spark.sql.dataframeCache.logLevel
+spark.sql.datetime.java8API.enabled
+spark.sql.debug
+spark.sql.debug.maxToStringFields
+spark.sql.decimalOperations.allowPrecisionLoss
+spark.sql.defaultCacheStorageLevel
+spark.sql.defaultCatalog
+spark.sql.defaultColumn.allowedProviders
+spark.sql.defaultColumn.enabled
+spark.sql.defaultColumn.useNullsForMissingDefaultValues
+spark.sql.defaultSizeInBytes
+spark.sql.defaultUrlStreamHandlerFactory.enabled
+spark.sql.dropTableOnView.enabled
+spark.sql.enforceTypeCoercionBeforeUnionDeduplication.enabled
+spark.sql.error.messageFormat
+spark.sql.event.truncate.length
+spark.sql.exchange.reuse
+spark.sql.execution.arrow.compression.codec
+spark.sql.execution.arrow.compression.zstd.level
+spark.sql.execution.arrow.enabled
+spark.sql.execution.arrow.fallback.enabled
+spark.sql.execution.arrow.localRelationThreshold
+spark.sql.execution.arrow.maxBytesPerBatch
+spark.sql.execution.arrow.maxBytesPerOutputBatch
+spark.sql.execution.arrow.maxRecordsPerBatch
+spark.sql.execution.arrow.maxRecordsPerOutputBatch
+spark.sql.execution.arrow.pyspark.enabled
+spark.sql.execution.arrow.pyspark.fallback.enabled
+spark.sql.execution.arrow.pyspark.selfDestruct.enabled
+spark.sql.execution.arrow.pyspark.validateSchema.enabled
+spark.sql.execution.arrow.sparkr.enabled
+spark.sql.execution.arrow.transformWithStateInPySpark.maxStateRecordsPerBatch
+spark.sql.execution.arrow.useLargeVarTypes
+spark.sql.execution.broadcastHashJoin.outputPartitioningExpandLimit
+spark.sql.execution.datasources.hadoopLineRecordReader.enabled
+spark.sql.execution.fastFailOnFileFormatOutput
+spark.sql.execution.interruptOnCancel
+spark.sql.execution.pandas.convertToArrowArraySafely
+spark.sql.execution.pandas.inferPandasDictAsMap
+spark.sql.execution.pandas.structHandlingMode
+spark.sql.execution.pandas.udf.buffer.size
+spark.sql.execution.pyspark.binaryAsBytes
+spark.sql.execution.pyspark.python
+spark.sql.execution.pyspark.udf.daemonKillWorkerOnFlushFailure
+spark.sql.execution.pyspark.udf.faulthandler.enabled
+spark.sql.execution.pyspark.udf.hideTraceback.enabled
+spark.sql.execution.pyspark.udf.idleTimeoutSeconds
+spark.sql.execution.pyspark.udf.killOnIdleTimeout
+spark.sql.execution.pyspark.udf.simplifiedTraceback.enabled
+spark.sql.execution.pyspark.udf.tracebackDumpIntervalSeconds
+spark.sql.execution.python.udf.buffer.size
+spark.sql.execution.python.udf.maxRecordsPerBatch
+spark.sql.execution.pythonUDF.arrow.concurrency.level
+spark.sql.execution.pythonUDF.arrow.enabled
+spark.sql.execution.pythonUDF.arrow.legacy.fallbackOnUDT
+spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled
+spark.sql.execution.pythonUDF.pandas.preferIntExtensionDtype
+spark.sql.execution.pythonUDTF.arrow.enabled
+spark.sql.execution.rangeExchange.sampleSizePerPartition
+spark.sql.execution.removeRedundantProjects
+spark.sql.execution.removeRedundantSorts
+spark.sql.execution.replaceHashWithSortAgg
+spark.sql.execution.reuseSubquery
+spark.sql.execution.sortBeforeRepartition
+spark.sql.execution.topKSortFallbackThreshold
+spark.sql.execution.useObjectHashAggregateExec
+spark.sql.execution.usePartitionEvaluator
+spark.sql.extendedExplainProviders
+spark.sql.extensions
+spark.sql.extensions.test.loadFromCp
+spark.sql.fileSource.insert.enforceNotNull
+spark.sql.files.ignoreCorruptFiles
+spark.sql.files.ignoreInvalidPartitionPaths
+spark.sql.files.ignoreMissingFiles
+spark.sql.files.maxPartitionBytes
+spark.sql.files.maxPartitionNum
+spark.sql.files.maxRecordsPerFile
+spark.sql.files.minPartitionNum
+spark.sql.files.openCostInBytes
+spark.sql.files.supportSecondOffsetFormat
+spark.sql.filesourceTableRelationCacheSize
+spark.sql.function.concatBinaryAsString
+spark.sql.function.eltOutputAsString
+spark.sql.function.protobufExtensions.enabled
+spark.sql.functionResolution.sessionOrder
+spark.sql.geospatial.enabled
+spark.sql.globalTempDatabase
+spark.sql.groupByAliases
+spark.sql.groupByOrdinal
+spark.sql.hive.advancedPartitionPredicatePushdown.enabled
+spark.sql.hive.caseSensitiveInferenceMode
+spark.sql.hive.convertCTAS
+spark.sql.hive.convertMetastoreAsNullable
+spark.sql.hive.convertMetastoreInsertDir
+spark.sql.hive.convertMetastoreParquet.mergeSchema
+spark.sql.hive.dropPartitionByName.enabled
+spark.sql.hive.filesourcePartitionFileCacheSize
+spark.sql.hive.gatherFastStats
+spark.sql.hive.manageFilesourcePartitions
+spark.sql.hive.metastore.barrierPrefixes
+spark.sql.hive.metastore.jars
+spark.sql.hive.metastore.jars.path
+spark.sql.hive.metastore.sharedPrefixes
+spark.sql.hive.metastore.version
+spark.sql.hive.metastorePartitionPruning
+spark.sql.hive.metastorePartitionPruningFallbackOnException
+spark.sql.hive.metastorePartitionPruningFastFallback
+spark.sql.hive.metastorePartitionPruningInSetThreshold
+spark.sql.hive.tablePropertyLengthThreshold
+spark.sql.hive.thriftServer.async
+spark.sql.hive.thriftServer.singleSession
+spark.sql.hive.useDelegateForSymlinkTextInputFormat
+spark.sql.hive.version
+spark.sql.icu.caseMappings.enabled
+spark.sql.inMemoryColumnarStorage.batchSize
+spark.sql.inMemoryColumnarStorage.compressed
+spark.sql.inMemoryColumnarStorage.enableVectorizedReader
+spark.sql.inMemoryColumnarStorage.hugeVectorReserveRatio
+spark.sql.inMemoryColumnarStorage.hugeVectorThreshold
+spark.sql.inMemoryColumnarStorage.partitionPruning
+spark.sql.inMemoryTableScanStatistics.enable
+spark.sql.join.preferSortMergeJoin
+spark.sql.json.enableExactStringParsing
+spark.sql.json.enablePartialResults
+spark.sql.json.filterPushdown.enabled
+spark.sql.json.useUnsafeRow
+spark.sql.jsonGenerator.ignoreNullFields
+spark.sql.jsonGenerator.writeNullIfWithDefaultValue
+spark.sql.lateralColumnAlias.enableImplicitResolution
+spark.sql.lazySetOperatorOutput.enabled
+spark.sql.leafNodeDefaultParallelism
+spark.sql.legacy.addSingleFileInAddFile
+spark.sql.legacy.allowAutoGeneratedAliasForView
+spark.sql.legacy.allowEmptySchemaWrite
+spark.sql.legacy.allowHashOnMapType
+spark.sql.legacy.allowNegativeScaleOfDecimal
+spark.sql.legacy.allowNonEmptyLocationInCTAS
+spark.sql.legacy.allowNullComparisonResultInArraySort
+spark.sql.legacy.allowParameterlessCount
+spark.sql.legacy.allowSessionVariableInPersistedView
+spark.sql.legacy.allowStarWithSingleTableIdentifierInCount
+spark.sql.legacy.allowTempViewCreationWithMultipleNameparts
+spark.sql.legacy.allowUntypedScalaUDF
+spark.sql.legacy.allowZeroIndexInFormatString
+spark.sql.legacy.avro.allowIncompatibleSchema
+spark.sql.legacy.bangEqualsNot
+spark.sql.legacy.blockCreateTempTableUsingProvider
+spark.sql.legacy.bucketedTableScan.outputOrdering
+spark.sql.legacy.castComplexTypesToString.enabled
+spark.sql.legacy.charVarcharAsString
+spark.sql.legacy.codingErrorAction
+spark.sql.legacy.collationAwareHashFunctions
+spark.sql.legacy.consecutiveStringLiterals.enabled
+spark.sql.legacy.createEmptyCollectionUsingStringType
+spark.sql.legacy.createHiveTableByDefault
+spark.sql.legacy.csv.enableDateTimeParsingFallback
+spark.sql.legacy.cteDuplicateAttributeNames
+spark.sql.legacy.ctePrecedencePolicy
+spark.sql.legacy.dataFrameWriterV2IgnorePathOption
+spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue
+spark.sql.legacy.db2.booleanMapping.enabled
+spark.sql.legacy.db2.numericMapping.enabled
+spark.sql.legacy.decimal.retainFractionDigitsOnTruncate
+spark.sql.legacy.disableMapKeyNormalization
+spark.sql.legacy.doLooseUpcast
+spark.sql.legacy.duplicateBetweenInput
+spark.sql.legacy.earlyEvalCurrentTime
+spark.sql.legacy.emptyCurrentDBInCli
+spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName
+spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled
+spark.sql.legacy.execution.pythonUDTF.pandas.conversion.enabled
+spark.sql.legacy.exponentLiteralAsDecimal.enabled
+spark.sql.legacy.extraOptionsBehavior.enabled
+spark.sql.legacy.followThreeValuedLogicInArrayExists
+spark.sql.legacy.fromDayTimeString.enabled
+spark.sql.legacy.groupingIdWithAppendedUserGroupBy
+spark.sql.legacy.histogramNumericPropagateInputType
+spark.sql.legacy.hive.thriftServer.useZeroBasedColumnOrdinalPosition
+spark.sql.legacy.identifierClause
+spark.sql.legacy.inSubqueryNullability
+spark.sql.legacy.inlineCTEInCommands
+spark.sql.legacy.integerGroupingId
+spark.sql.legacy.interval.enabled
+spark.sql.legacy.javaCharsets
+spark.sql.legacy.json.allowEmptyString.enabled
+spark.sql.legacy.json.enableDateTimeParsingFallback
+spark.sql.legacy.keepCommandOutputSchema
+spark.sql.legacy.keepPartitionSpecAsStringLiteral
+spark.sql.legacy.literal.pickMinimumPrecision
+spark.sql.legacy.lpadRpadAlwaysReturnString
+spark.sql.legacy.mssqlserver.datetimeoffsetMapping.enabled
+spark.sql.legacy.mssqlserver.numericMapping.enabled
+spark.sql.legacy.mysql.bitArrayMapping.enabled
+spark.sql.legacy.mysql.timestampNTZMapping.enabled
+spark.sql.legacy.negativeIndexInArrayInsert
+spark.sql.legacy.noCharPaddingInPredicate
+spark.sql.legacy.notReserveProperties
+spark.sql.legacy.nullInEmptyListBehavior
+spark.sql.legacy.nullValueWrittenAsQuotedEmptyStringCsv
+spark.sql.legacy.oracle.timestampMapping.enabled
+spark.sql.legacy.parameterSubstitution.constantsOnly
+spark.sql.legacy.parquet.nanosAsLong
+spark.sql.legacy.parquet.returnNullStructIfAllFieldsMissing
+spark.sql.legacy.parseNullPartitionSpecAsStringLiteral
+spark.sql.legacy.parseQueryWithoutEof
+spark.sql.legacy.parser.havingWithoutGroupByAsWhere
+spark.sql.legacy.pathOptionBehavior.enabled
+spark.sql.legacy.percentileDiscCalculation
+spark.sql.legacy.postgres.datetimeMapping.enabled
+spark.sql.legacy.raiseErrorWithoutErrorClass
+spark.sql.legacy.readFileSourceTableCacheIgnoreOptions
+spark.sql.legacy.replaceDatabricksSparkAvro.enabled
+spark.sql.legacy.respectNullabilityInTextDatasetConversion
+spark.sql.legacy.scalarSubqueryAllowGroupByNonEqualityCorrelatedPredicate
+spark.sql.legacy.scalarSubqueryCountBugBehavior
+spark.sql.legacy.sessionInitWithConfigDefaults
+spark.sql.legacy.setCommandRejectsSparkCoreConfs
+spark.sql.legacy.setopsPrecedence.enabled
+spark.sql.legacy.sizeOfNull
+spark.sql.legacy.skipTypeValidationOnAlterPartition
+spark.sql.legacy.statisticalAggregate
+spark.sql.legacy.storeAnalyzedPlanForView
+spark.sql.legacy.timeParserPolicy
+spark.sql.legacy.truncateForEmptyRegexSplit
+spark.sql.legacy.typeCoercion.datetimeToString.enabled
+spark.sql.legacy.useCurrentConfigsForView
+spark.sql.legacy.useLegacyXMLParser
+spark.sql.legacy.useV1Command
+spark.sql.legacy.v1IdentifierNoCatalog
+spark.sql.legacy.viewSchemaBindingMode
+spark.sql.legacy.viewSchemaCompensation
+spark.sql.lightweightPlanChangeValidation
+spark.sql.limit.initialNumPartitions
+spark.sql.limit.scaleUpFactor
+spark.sql.listagg.allowDistinctCastWithOrder.enabled
+spark.sql.mapKeyDedupPolicy
+spark.sql.mapZipWithUsesJavaCollections
+spark.sql.maven.additionalRemoteRepositories
+spark.sql.maxBroadcastTableSize
+spark.sql.maxConcurrentOutputFileWriters
+spark.sql.maxMetadataStringLength
+spark.sql.maxPlanStringLength
+spark.sql.maxSinglePartitionBytes
+spark.sql.mergeNestedTypeCoercion.enabled
+spark.sql.metadataCacheTTLSeconds
+spark.sql.nameResolutionLog.level
+spark.sql.objectHashAggregate.sortBased.fallbackThreshold
+spark.sql.operatorPipeSyntaxEnabled
+spark.sql.optimizeNullAwareAntiJoin
+spark.sql.optimizer.avoidCollapseUDFWithExpensiveExpr
+spark.sql.optimizer.avoidDoubleFilterEval
+spark.sql.optimizer.canChangeCachedPlanOutputPartitioning
+spark.sql.optimizer.collapseProjectAlwaysInline
+spark.sql.optimizer.datasourceV2ExprFolding
+spark.sql.optimizer.datasourceV2JoinPushdown
+spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled
+spark.sql.optimizer.decorrelateInnerQuery.enabled
+spark.sql.optimizer.decorrelateJoinPredicate.enabled
+spark.sql.optimizer.decorrelateLimit.enabled
+spark.sql.optimizer.decorrelateOffset.enabled
+spark.sql.optimizer.decorrelatePredicateSubqueriesInJoinPredicate.enabled
+spark.sql.optimizer.decorrelateSetOps.enabled
+spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled
+spark.sql.optimizer.decorrelateSubqueryPreventConstantHoldingForCountBug.enabled
+spark.sql.optimizer.decorrelateUnionOrSetOpUnderLimit.enabled
+spark.sql.optimizer.disableHints
+spark.sql.optimizer.dynamicPartitionPruning.enabled
+spark.sql.optimizer.dynamicPartitionPruning.fallbackFilterRatio
+spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly
+spark.sql.optimizer.dynamicPartitionPruning.useStats
+spark.sql.optimizer.enableCsvExpressionOptimization
+spark.sql.optimizer.enableJsonExpressionOptimization
+spark.sql.optimizer.excludeSubqueryRefsFromRemoveRedundantAliases.enabled
+spark.sql.optimizer.excludedRules
+spark.sql.optimizer.expression.nestedPruning.enabled
+spark.sql.optimizer.expressionProjectionCandidateLimit
+spark.sql.optimizer.inSetConversionThreshold
+spark.sql.optimizer.inSetSwitchThreshold
+spark.sql.optimizer.maxIterations
+spark.sql.optimizer.metadataOnly
+spark.sql.optimizer.nestedPredicatePushdown.supportedFileSources
+spark.sql.optimizer.nestedSchemaPruning.enabled
+spark.sql.optimizer.optimizeOneRowRelationSubquery
+spark.sql.optimizer.optimizeOneRowRelationSubquery.alwaysInline
+spark.sql.optimizer.optimizeUncorrelatedInSubqueriesInJoinCondition.enabled
+spark.sql.optimizer.plannedWrite.enabled
+spark.sql.optimizer.preserveAliasMetadataWhenCollapsingProjects
+spark.sql.optimizer.propagateDistinctKeys.enabled
+spark.sql.optimizer.pruneFiltersCanPruneStreamingSubplan
+spark.sql.optimizer.pullHintsIntoSubqueries
+spark.sql.optimizer.pullOutNestedDataOuterRefExpressions.enabled
+spark.sql.optimizer.replaceExceptWithFilter
+spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold
+spark.sql.optimizer.runtime.bloomFilter.creationSideThreshold
+spark.sql.optimizer.runtime.bloomFilter.enabled
+spark.sql.optimizer.runtime.bloomFilter.expectedNumItems
+spark.sql.optimizer.runtime.bloomFilter.maxNumBits
+spark.sql.optimizer.runtime.bloomFilter.maxNumItems
+spark.sql.optimizer.runtime.bloomFilter.numBits
+spark.sql.optimizer.runtime.rowLevelOperationGroupFilter.enabled
+spark.sql.optimizer.runtimeFilter.number.threshold
+spark.sql.optimizer.scalarSubqueryUseSingleJoin
+spark.sql.optimizer.serializer.nestedSchemaPruning.enabled
+spark.sql.optimizer.windowGroupLimitThreshold
+spark.sql.optimizer.wrapExistsInAggregateFunction
+spark.sql.orc.aggregatePushdown
+spark.sql.orc.columnarReaderBatchSize
+spark.sql.orc.columnarWriterBatchSize
+spark.sql.orc.compression.codec
+spark.sql.orc.enableNestedColumnVectorizedReader
+spark.sql.orc.enableVectorizedReader
+spark.sql.orc.filterPushdown
+spark.sql.orc.impl
+spark.sql.orc.mergeSchema
+spark.sql.orderByOrdinal
+spark.sql.orderingAwareLimitOffset
+spark.sql.parquet.aggregatePushdown
+spark.sql.parquet.binaryAsString
+spark.sql.parquet.columnarReaderBatchSize
+spark.sql.parquet.compression.codec
+spark.sql.parquet.datetimeRebaseModeInRead
+spark.sql.parquet.datetimeRebaseModeInWrite
+spark.sql.parquet.enableNestedColumnVectorizedReader
+spark.sql.parquet.enableNullTypeVectorizedReader
+spark.sql.parquet.enableVectorizedReader
+spark.sql.parquet.fieldId.read.enabled
+spark.sql.parquet.fieldId.read.ignoreMissing
+spark.sql.parquet.fieldId.write.enabled
+spark.sql.parquet.filterPushdown
+spark.sql.parquet.filterPushdown.date
+spark.sql.parquet.filterPushdown.decimal
+spark.sql.parquet.filterPushdown.string.startsWith
+spark.sql.parquet.filterPushdown.stringPredicate
+spark.sql.parquet.filterPushdown.timestamp
+spark.sql.parquet.ignoreVariantAnnotation
+spark.sql.parquet.inferTimestampNTZ.enabled
+spark.sql.parquet.int96AsTimestamp
+spark.sql.parquet.int96RebaseModeInRead
+spark.sql.parquet.int96RebaseModeInWrite
+spark.sql.parquet.int96TimestampConversion
+spark.sql.parquet.mergeSchema
+spark.sql.parquet.output.committer.class
+spark.sql.parquet.outputTimestampType
+spark.sql.parquet.pushdown.inFilterThreshold
+spark.sql.parquet.recordLevelFilter.enabled
+spark.sql.parquet.respectSummaryFiles
+spark.sql.parquet.variant.annotateLogicalType.enabled
+spark.sql.parquet.writeLegacyFormat
+spark.sql.parser.eagerEvalOfUnresolvedInlineTable
+spark.sql.parser.escapedStringLiterals
+spark.sql.parser.manageParserCaches
+spark.sql.parser.parserDfaCacheFlushRatio
+spark.sql.parser.parserDfaCacheFlushThreshold
+spark.sql.parser.quotedRegexColumnNames
+spark.sql.parser.singleCharacterPipeOperator.enabled
+spark.sql.pipelines.event.queue.capacity
+spark.sql.pipelines.execution.maxConcurrentFlows
+spark.sql.pipelines.execution.streamstate.pollingInterval
+spark.sql.pipelines.execution.watchdog.maxRetryTime
+spark.sql.pipelines.execution.watchdog.minRetryTime
+spark.sql.pipelines.maxFlowRetryAttempts
+spark.sql.pipelines.timeoutMsForTerminationJoinAndLock
+spark.sql.pivotMaxValues
+spark.sql.planChangeLog.batches
+spark.sql.planChangeLog.rules
+spark.sql.planChangeValidation
+spark.sql.planner.pythonExecution.memory
+spark.sql.preserveCharVarcharTypeInfo
+spark.sql.prioritizeOrdinalResolutionInSort.enabled
+spark.sql.pyspark.dataSource.profiler
+spark.sql.pyspark.inferNestedDictAsStruct.enabled
+spark.sql.pyspark.jvmStacktrace.enabled
+spark.sql.pyspark.legacy.inferArrayTypeFromFirstElement.enabled
+spark.sql.pyspark.legacy.inferMapTypeFromFirstPair.enabled
+spark.sql.pyspark.plotting.max_rows
+spark.sql.pyspark.toJSON.returnDataFrame
+spark.sql.pyspark.udf.profiler
+spark.sql.pyspark.worker.logging.enabled
+spark.sql.python.filterPushdown.enabled
+spark.sql.queryExecutionListeners
+spark.sql.readSideCharPadding
+spark.sql.redaction.options.regex
+spark.sql.redaction.string.regex
+spark.sql.repl.eagerEval.enabled
+spark.sql.repl.eagerEval.maxNumRows
+spark.sql.repl.eagerEval.truncate
+spark.sql.requireAllClusterKeysForCoPartition
+spark.sql.requireAllClusterKeysForDistribution
+spark.sql.resultQueryStage.maxThreadThreshold
+spark.sql.retainGroupColumns
+spark.sql.runCollationTypeCastsBeforeAliasAssignment.enabled
+spark.sql.runSQLOnFiles
+spark.sql.scriptTransformation.exitTimeoutInSeconds
+spark.sql.scripting.continueHandlerEnabled
+spark.sql.scripting.cursorEnabled
+spark.sql.scripting.enabled
+spark.sql.selfJoinAutoResolveAmbiguity
+spark.sql.session.localRelationBatchOfChunksSizeBytes
+spark.sql.session.localRelationCacheThreshold
+spark.sql.session.localRelationChunkSizeBytes
+spark.sql.session.localRelationChunkSizeLimit
+spark.sql.session.localRelationChunkSizeRows
+spark.sql.session.localRelationSizeLimit
+spark.sql.session.timeZone
+spark.sql.sessionWindow.buffer.in.memory.threshold
+spark.sql.sessionWindow.buffer.spill.size.threshold
+spark.sql.sessionWindow.buffer.spill.threshold
+spark.sql.shuffle.orderIndependentChecksum.enableFullRetryOnMismatch
+spark.sql.shuffle.orderIndependentChecksum.enableQueryLevelRollbackOnMismatch
+spark.sql.shuffle.orderIndependentChecksum.enabled
+spark.sql.shuffle.partitions
+spark.sql.shuffleDependency.fileCleanup.enabled
+spark.sql.shuffleDependency.skipMigration.enabled
+spark.sql.shuffleExchange.maxThreadThreshold
+spark.sql.shuffledHashJoinFactor
+spark.sql.sort.enableRadixSort
+spark.sql.sortMergeJoinExec.buffer.in.memory.threshold
+spark.sql.sortMergeJoinExec.buffer.spill.size.threshold
+spark.sql.sortMergeJoinExec.buffer.spill.threshold
+spark.sql.sources.binaryFile.maxLength
+spark.sql.sources.bucketing.autoBucketedScan.enabled
+spark.sql.sources.bucketing.enabled
+spark.sql.sources.bucketing.maxBuckets
+spark.sql.sources.commitProtocolClass
+spark.sql.sources.default
+spark.sql.sources.disabledJdbcConnProviderList
+spark.sql.sources.fileCompressionFactor
+spark.sql.sources.ignoreDataLocality
+spark.sql.sources.outputCommitterClass
+spark.sql.sources.parallelPartitionDiscovery.parallelism
+spark.sql.sources.parallelPartitionDiscovery.threshold
+spark.sql.sources.partitionColumnTypeInference.enabled
+spark.sql.sources.partitionOverwriteMode
+spark.sql.sources.schemaStringLengthThreshold
+spark.sql.sources.useListFilesFileSystemList
+spark.sql.sources.useV1SourceList
+spark.sql.sources.v2.bucketing.allowCompatibleTransforms.enabled
+spark.sql.sources.v2.bucketing.allowJoinKeysSubsetOfPartitionKeys.enabled
+spark.sql.sources.v2.bucketing.enabled
+spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled
+spark.sql.sources.v2.bucketing.partition.filter.enabled
+spark.sql.sources.v2.bucketing.pushPartValues.enabled
+spark.sql.sources.v2.bucketing.shuffle.enabled
+spark.sql.sources.v2.bucketing.sorting.enabled
+spark.sql.sources.validatePartitionColumns
+spark.sql.stableDerivedColumnAlias.enabled
+spark.sql.stackTracesInDataFrameContext
+spark.sql.statistics.fallBackToHdfs
+spark.sql.statistics.histogram.enabled
+spark.sql.statistics.histogram.numBins
+spark.sql.statistics.ndv.maxError
+spark.sql.statistics.parallelFileListingInStatsComputation.enabled
+spark.sql.statistics.percentile.accuracy
+spark.sql.statistics.size.autoUpdate.enabled
+spark.sql.statistics.updatePartitionStatsInAnalyzeTable.enabled
+spark.sql.storeAssignmentPolicy
+spark.sql.streaming.aggregation.stateFormatVersion
+spark.sql.streaming.asyncLogPurge.enabled
+spark.sql.streaming.checkUnfinishedRepartitionOnRestart
+spark.sql.streaming.checkpoint.escapedPathCheck.enabled
+spark.sql.streaming.checkpoint.fileChecksum.enabled
+spark.sql.streaming.checkpoint.fileChecksum.skipCreationIfFileMissingChecksum
+spark.sql.streaming.checkpoint.renamedFileCheck.enabled
+spark.sql.streaming.checkpoint.verifyMetadataExists.enabled
+spark.sql.streaming.checkpointLocation
+spark.sql.streaming.commitProtocolClass
+spark.sql.streaming.continuous.epochBacklogQueueSize
+spark.sql.streaming.continuous.executorPollIntervalMs
+spark.sql.streaming.continuous.executorQueueSize
+spark.sql.streaming.disabledV2MicroBatchReaders
+spark.sql.streaming.disabledV2Writers
+spark.sql.streaming.fileSink.log.cleanupDelay
+spark.sql.streaming.fileSink.log.compactInterval
+spark.sql.streaming.fileSink.log.deletion
+spark.sql.streaming.fileSource.cleaner.numThreads
+spark.sql.streaming.fileSource.log.cleanupDelay
+spark.sql.streaming.fileSource.log.compactInterval
+spark.sql.streaming.fileSource.log.deletion
+spark.sql.streaming.fileSource.schema.forceNullable
+spark.sql.streaming.fileStreamSink.ignoreMetadata
+spark.sql.streaming.flatMapGroupsWithState.skipEmittingInitialStateKeys
+spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion
+spark.sql.streaming.forceDeleteTempCheckpointLocation
+spark.sql.streaming.internal.stateStore.partitions
+spark.sql.streaming.join.stateFormatVersion
+spark.sql.streaming.kafka.useDeprecatedOffsetFetching
+spark.sql.streaming.maxBatchesToRetainInMemory
+spark.sql.streaming.metadataCache.enabled
+spark.sql.streaming.metricsEnabled
+spark.sql.streaming.minBatchesToRetain
+spark.sql.streaming.multipleWatermarkPolicy
+spark.sql.streaming.noDataMicroBatches.enabled
+spark.sql.streaming.noDataProgressEventInterval
+spark.sql.streaming.numRecentProgressUpdates
+spark.sql.streaming.offsetLog.formatVersion
+spark.sql.streaming.optimizeOneRowPlan.enabled
+spark.sql.streaming.pollingDelay
+spark.sql.streaming.queryEvolution.enableSourceEvolution
+spark.sql.streaming.ratioExtraSpaceAllowedInCheckpoint
+spark.sql.streaming.realTimeMode.allowlistCheck
+spark.sql.streaming.realTimeMode.minBatchDuration
+spark.sql.streaming.schemaInference
+spark.sql.streaming.sessionWindow.merge.sessions.in.local.partition
+spark.sql.streaming.sessionWindow.stateFormatVersion
+spark.sql.streaming.stateStore.autoSnapshotRepair.enabled
+spark.sql.streaming.stateStore.autoSnapshotRepair.maxChangeFileReplay
+spark.sql.streaming.stateStore.autoSnapshotRepair.numFailuresBeforeActivating
+spark.sql.streaming.stateStore.checkpointFormatVersion
+spark.sql.streaming.stateStore.commitValidation.enabled
+spark.sql.streaming.stateStore.compression.codec
+spark.sql.streaming.stateStore.coordinatorReportSnapshotUploadLag
+spark.sql.streaming.stateStore.encodingFormat
+spark.sql.streaming.stateStore.forceSnapshotUploadOnLag
+spark.sql.streaming.stateStore.formatValidation.enabled
+spark.sql.streaming.stateStore.maintenanceForceShutdownTimeout
+spark.sql.streaming.stateStore.maintenanceInterval
+spark.sql.streaming.stateStore.maintenanceProcessingTimeout
+spark.sql.streaming.stateStore.maintenanceShutdownTimeout
+spark.sql.streaming.stateStore.maxLaggingStoresToReport
+spark.sql.streaming.stateStore.maxNumStateSchemaFiles
+spark.sql.streaming.stateStore.maxVersionsToDeletePerMaintenance
+spark.sql.streaming.stateStore.minDeltasForSnapshot
+spark.sql.streaming.stateStore.multiplierForMinTimeDiffToLog
+spark.sql.streaming.stateStore.multiplierForMinVersionDiffToLog
+spark.sql.streaming.stateStore.numStateStoreInstanceMetricsToReport
+spark.sql.streaming.stateStore.numStateStoreMaintenanceThreads
+spark.sql.streaming.stateStore.providerClass
+spark.sql.streaming.stateStore.rocksdb.formatVersion
+spark.sql.streaming.stateStore.rocksdb.mergeOperatorVersion
+spark.sql.streaming.stateStore.rowChecksum.enabled
+spark.sql.streaming.stateStore.rowChecksum.readVerificationRatio
+spark.sql.streaming.stateStore.skipNullsForStreamStreamJoins.enabled
+spark.sql.streaming.stateStore.snapshotLagReportInterval
+spark.sql.streaming.stateStore.stateSchemaCheck
+spark.sql.streaming.stateStore.unloadOnCommit
+spark.sql.streaming.stateStore.valueStateSchemaEvolutionThreshold
+spark.sql.streaming.statefulOperator.allowMultiple
+spark.sql.streaming.statefulOperator.checkCorrectness.enabled
+spark.sql.streaming.statefulOperator.useStrictDistribution
+spark.sql.streaming.stopActiveRunOnRestart
+spark.sql.streaming.stopTimeout
+spark.sql.streaming.streamingQueryListeners
+spark.sql.streaming.transformWithState.stateSchemaVersion
+spark.sql.streaming.triggerAvailableNowWrapper.enabled
+spark.sql.streaming.ui.enabled
+spark.sql.streaming.ui.enabledCustomMetricList
+spark.sql.streaming.ui.retainedProgressUpdates
+spark.sql.streaming.ui.retainedQueries
+spark.sql.streaming.unsupportedOperationCheck
+spark.sql.streaming.validateEventTimeWatermarkColumn
+spark.sql.streaming.verifyCheckpointDirectoryEmptyOnStart
+spark.sql.subexpressionElimination.cache.maxEntries
+spark.sql.subexpressionElimination.enabled
+spark.sql.subexpressionElimination.skipForShortcutExpr
+spark.sql.subquery.maxThreadThreshold
+spark.sql.thriftServer.incrementalCollect
+spark.sql.thriftServer.interruptOnCancel
+spark.sql.thriftServer.queryTimeout
+spark.sql.thriftserver.scheduler.pool
+spark.sql.thriftserver.shuffleDependency.fileCleanup.enabled
+spark.sql.thriftserver.ui.retainedSessions
+spark.sql.thriftserver.ui.retainedStatements
+spark.sql.timeTravelTimestampKey
+spark.sql.timeTravelVersionKey
+spark.sql.timeType.enabled
+spark.sql.timestampType
+spark.sql.transposeMaxValues
+spark.sql.truncateTable.ignorePermissionAcl.enabled
+spark.sql.tvf.allowMultipleTableArguments.enabled
+spark.sql.types.framework.enabled
+spark.sql.ui.explainMode
+spark.sql.ui.retainedExecutions
+spark.sql.unionOutputPartitioning
+spark.sql.useCommonExprIdForAlias
+spark.sql.variable.substitute
+spark.sql.variant.allowDuplicateKeys
+spark.sql.variant.allowReadingShredded
+spark.sql.variant.forceShreddingSchemaForTest
+spark.sql.variant.inferShreddingSchema
+spark.sql.variant.pushVariantIntoScan
+spark.sql.variant.shredding.maxSchemaDepth
+spark.sql.variant.shredding.maxSchemaWidth
+spark.sql.variant.writeShredding.enabled
+spark.sql.view.maxNestedViewDepth
+spark.sql.warehouse.dir
+spark.sql.windowExec.buffer.in.memory.threshold
+spark.sql.windowExec.buffer.spill.size.threshold
+spark.sql.windowExec.buffer.spill.threshold
+spark.stage.ignoreDecommissionFetchFailure
+spark.stage.maxAttempts
+spark.stage.maxConsecutiveAttempts
+spark.standalone.executorRemoveDelayOnDisconnection
+spark.standalone.submit.waitAppCompletion
+spark.storage.blockManagerHeartbeatTimeoutMs
+spark.storage.blockManagerMasterDriverHeartbeatTimeoutMs
+spark.storage.blockManagerTimeoutIntervalMs
+spark.storage.cachedPeersTtl
+spark.storage.cleanupFilesAfterExecutorExit
+spark.storage.decommission.enabled
+spark.storage.decommission.fallbackStorage.cleanUp
+spark.storage.decommission.fallbackStorage.path
+spark.storage.decommission.maxReplicationFailuresPerBlock
+spark.storage.decommission.rddBlocks.enabled
+spark.storage.decommission.replicationReattemptInterval
+spark.storage.decommission.shuffleBlocks.enabled
+spark.storage.decommission.shuffleBlocks.maxDiskSize
+spark.storage.decommission.shuffleBlocks.maxThreads
+spark.storage.exceptionOnPinLeak
+spark.storage.localDiskByExecutors.cacheSize
+spark.storage.maxReplicationFailures
+spark.storage.memoryMapLimitForTests
+spark.storage.memoryMapThreshold
+spark.storage.replication.policy
+spark.storage.replication.proactive
+spark.storage.replication.topologyFile
+spark.storage.replication.topologyMapper
+spark.storage.shuffleManager.initWaitingTimeout
+spark.storage.unrollMemoryCheckPeriod
+spark.storage.unrollMemoryGrowthFactor
+spark.storage.unrollMemoryThreshold
+spark.submit.callSystemExitOnMainExit
+spark.submit.deployMode
+spark.submit.proxyUser.allowCustomClasspathInClusterMode
+spark.submit.pyFiles
+spark.task.cpus
+spark.task.maxDirectResultSize
+spark.task.maxFailures
+spark.task.reaper.enabled
+spark.task.reaper.killTimeout
+spark.task.reaper.pollingInterval
+spark.task.reaper.threadDump
+spark.taskMetrics.trackUpdatedBlockStatuses
+spark.test.noStageRetry
+spark.testing
+spark.testing.dynamicAllocation.schedule.enabled
+spark.testing.memory
+spark.testing.nCoresPerExecutor
+spark.testing.nExecutorsPerHost
+spark.testing.nHosts
+spark.testing.reservedMemory
+spark.testing.resourceProfileManager
+spark.testing.skipESSRegister
+spark.testing.skipValidateCores
+spark.ui.allowFramingFrom
+spark.ui.consoleProgress.update.interval
+spark.ui.custom.executor.log.url
+spark.ui.dagGraph.retainedRootRDDs
+spark.ui.enabled
+spark.ui.filters
+spark.ui.groupSQLSubExecutionEnabled
+spark.ui.heapHistogramEnabled
+spark.ui.jettyStopTimeout
+spark.ui.killEnabled
+spark.ui.liveUpdate.minFlushPeriod
+spark.ui.liveUpdate.period
+spark.ui.port
+spark.ui.prometheus.enabled
+spark.ui.proxyRedirectUri
+spark.ui.requestHeaderSize
+spark.ui.retainedDeadExecutors
+spark.ui.retainedJobs
+spark.ui.retainedStages
+spark.ui.retainedTasks
+spark.ui.reverseProxy
+spark.ui.reverseProxyUrl
+spark.ui.showConsoleProgress
+spark.ui.showErrorStacks
+spark.ui.store.path
+spark.ui.strictTransportSecurity
+spark.ui.threadDump.flamegraphEnabled
+spark.ui.threadDumpsEnabled
+spark.ui.timeline.executors.maximum
+spark.ui.timeline.jobs.maximum
+spark.ui.timeline.stages.maximum
+spark.ui.timeline.tasks.maximum
+spark.ui.timelineEnabled
+spark.ui.view.acls
+spark.ui.view.acls.groups
+spark.ui.xContentTypeOptions.enabled
+spark.ui.xXssProtection
+spark.unsafe.exceptionOnMemoryLeak
+spark.unsafe.sorter.spill.read.ahead.enabled
+spark.unsafe.sorter.spill.reader.buffer.size
+spark.user.groups.mapping
+spark.worker.executorStateSync.maxAttempts
+spark.yarn.dist.forceDownloadSchemes
+spark.yarn.dist.pyFiles
+spark.yarn.isPython
+spark.yarn.kerberos.renewal.excludeHadoopFileSystems
+spark.yarn.shuffle.server.recovery.disabled
+spark.yarn.stagingDir
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/configaudit/SparkConfigBindingPolicySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/configaudit/SparkConfigBindingPolicySuite.scala
new file mode 100644
index 000000000000..7b04db0788bd
--- /dev/null
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/configaudit/SparkConfigBindingPolicySuite.scala
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.configaudit
+
+import scala.io.Source
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.internal.config.{ConfigBindingPolicy, ConfigEntry}
+import org.apache.spark.sql.hive.HiveUtils
+import org.apache.spark.sql.internal.SQLConf
+
+class SparkConfigBindingPolicySuite extends SparkFunSuite {
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    // Ensure HiveUtils configs are registered before running tests.
+    // Accessing a field triggers the Scala object's static initializer.
+    assert(HiveUtils.CONVERT_METASTORE_PARQUET != null)
+  }
+
+  test("Test adding bindingPolicy to config") {
+    val allConfigs = SQLConf.getConfigEntries().asScala.filter { entry =>
+      entry.key == SQLConf.VIEW_SCHEMA_EVOLUTION_PRESERVE_USER_COMMENTS.key
+    }
+    assert(allConfigs.head.bindingPolicy.isDefined)
+    assert(allConfigs.head.bindingPolicy.get == ConfigBindingPolicy.SESSION)
+  }
+
+  test("Config enforcement for bindingPolicy") {
+    val allConfigsWithoutBindingPolicy: Iterable[ConfigEntry[_]] =
+      ConfigEntry.listAllEntries().asScala.filter { entry =>
+        entry.bindingPolicy.isEmpty
+      }
+    val filePath = getClass.getClassLoader.getResource(
+      
"conf/binding-policy-exceptions/configs-without-binding-policy-exceptions").getFile
+    val allowedNonViewInheritConfs: Set[String] = 
Source.fromFile(filePath).getLines().toSet
+    val missingBindingPolicyConfigs = allConfigsWithoutBindingPolicy.filterNot 
{ entry =>
+      allowedNonViewInheritConfs.contains(entry.key)
+    }.map(_.key).toList.sorted
+
+    if (missingBindingPolicyConfigs.nonEmpty) {
+      fail(
+        s"The following configs do not have bindingPolicy field set. You need 
to define it " +
+        "by using .withBindingPolicy(ConfigBindingPolicy.SESSION/PERSISTED) 
when you build " +
+        "the config entry.\n" +
+        missingBindingPolicyConfigs.mkString("\n")
+      )
+    }
+
+    val allConfigsWithBindingPolicy: Iterable[ConfigEntry[_]] =
+      SQLConf.getConfigEntries().asScala.filter { entry =>
+        entry.bindingPolicy.isDefined
+      }
+    allConfigsWithBindingPolicy.foreach { entry =>
+      if (allowedNonViewInheritConfs.contains(entry.key)) {
+        fail(
+          s"${entry.key} already has bindingPolicy set but still in the 
allowlist. You " +
+          s"should remove ${entry.key} from " +
+          "sql/hive/src/test/resources/conf/binding-policy-exceptions/" +
+          "configs-without-binding-policy-exceptions"
+        )
+      }
+    }
+  }
+
+  test("configs-without-binding-policy-exceptions file should be sorted 
alphabetically") {
+    val filePath = getClass.getClassLoader.getResource(
+      
"conf/binding-policy-exceptions/configs-without-binding-policy-exceptions").getFile
+    val allowedNonViewInheritConfs: Seq[String] = 
Source.fromFile(filePath).getLines().toSeq
+    val sortedAllowedNonViewInheritConfs: Seq[String] = 
allowedNonViewInheritConfs.sorted
+    if (allowedNonViewInheritConfs != sortedAllowedNonViewInheritConfs) {
+      fail("configs-without-binding-policy-exceptions file needs to be sorted 
alphabetically.")
+    }
+  }
+
+}


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

Reply via email to