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

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


The following commit(s) were added to refs/heads/master by this push:
     new f867059b3c0a fix(spark): SparkSQL write queries should correctly infer 
HUDI configs from spark.hoodie.* configs in spark conf (#18297)
f867059b3c0a is described below

commit f867059b3c0afb61fc75bbf3130075e4f979e29d
Author: Krishen <[email protected]>
AuthorDate: Tue Mar 10 10:45:12 2026 -0700

    fix(spark): SparkSQL write queries should correctly infer HUDI configs from 
spark.hoodie.* configs in spark conf (#18297)
    
    Co-authored-by: Krishen Bhan <“[email protected]”>
---
 .../spark/sql/hudi/HoodieSqlCommonUtils.scala      |  4 +++
 .../spark/sql/hudi/ProvidesHoodieConfig.scala      |  9 ++++--
 .../apache/spark/sql/hudi/common/TestSqlConf.scala | 35 +++++++++++++++++++++-
 3 files changed, 44 insertions(+), 4 deletions(-)

diff --git 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala
 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala
index 29368c8f02c7..451dd60d79de 100644
--- 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala
+++ 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala
@@ -248,6 +248,10 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport {
     opts.filterKeys(isHoodieConfigKey(_,
       opts.getOrElse(COMMIT_METADATA_KEYPREFIX.key, 
COMMIT_METADATA_KEYPREFIX.defaultValue()))).toMap
 
+  def extractSparkPrefixedHoodieConfigs(opts: Map[String, String]): 
Map[String, String] =
+    opts.filter { case (k, _) => k.startsWith("spark.hoodie.") }
+      .map { case (k, v) => (k.stripPrefix("spark."), v) }
+
   /**
    * Checks whether Spark is using Hive as Session's Catalog
    */
diff --git 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala
 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala
index 48bb7cf20f30..7d8df0b366e8 100644
--- 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala
+++ 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala
@@ -43,7 +43,7 @@ import 
org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, L
 import org.apache.spark.sql.execution.datasources.FileStatusCache
 import org.apache.spark.sql.hive.HiveExternalCatalog
 import 
org.apache.spark.sql.hudi.HoodieOptionConfig.mapSqlOptionsToDataSourceWriteConfigs
-import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{filterHoodieConfigs, 
isUsingHiveCatalog}
+import 
org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{extractSparkPrefixedHoodieConfigs,
 filterHoodieConfigs, isUsingHiveCatalog}
 import org.apache.spark.sql.hudi.ProvidesHoodieConfig.{buildOverridingOpts, 
buildOverridingOptsForDelete, combineOptions, getPartitionPathFieldWriteConfig}
 import org.apache.spark.sql.hudi.command.SqlKeyGenerator
 import org.apache.spark.sql.internal.SQLConf
@@ -515,18 +515,21 @@ object ProvidesHoodieConfig {
     // NOTE: Properties are merged in the following order of priority (first 
has the highest priority, last has the
     //       lowest, which is inverse to the ordering in the code):
     //          1. (Extra) Option overrides
-    //          2. Spark SQL configs
+    //          2. Spark SQL configs (hoodie.* keys)
+    //          2b. Spark SQL configs (spark.hoodie.* keys, normalized to 
hoodie.*)
     //          3. Persisted Hudi's Table configs
     //          4. Table's properties in Spark Catalog
     //          5. Global DFS properties
     //          6. (Feature-specific) Default values
+    val allConfs = sqlConf.getAllConfs
     filterNullValues(defaultOpts) ++
       DFSPropertiesConfiguration.getGlobalProps.asScala.toMap ++
       // NOTE: Catalog table provided t/h `TBLPROPERTIES` clause might contain 
Spark SQL specific
       //       properties that need to be mapped into Hudi's conventional ones
       mapSqlOptionsToDataSourceWriteConfigs(catalogTable.catalogProperties) ++
       tableConfig.getProps.asScala.toMap ++
-      filterHoodieConfigs(sqlConf.getAllConfs) ++
+      extractSparkPrefixedHoodieConfigs(allConfs) ++
+      filterHoodieConfigs(allConfs) ++
       filterNullValues(overridingOpts)
   }
 
diff --git 
a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSqlConf.scala
 
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSqlConf.scala
index 0f0af4363cce..e9c71ef6d7b3 100644
--- 
a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSqlConf.scala
+++ 
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSqlConf.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.hudi.common
 
 import org.apache.hudi.DataSourceReadOptions._
-import org.apache.hudi.common.config.DFSPropertiesConfiguration
+import org.apache.hudi.common.config.{DFSPropertiesConfiguration, 
HoodieMetadataConfig}
 import org.apache.hudi.common.model.HoodieTableType
 import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
 import org.apache.hudi.common.testutils.HoodieTestUtils
@@ -104,6 +104,39 @@ class TestSqlConf extends HoodieSparkSqlTestBase with 
BeforeAndAfter {
     }
   }
 
+  test("Test spark.hoodie.* configs propagate to write path") {
+    withTempDir { tmp =>
+      val tableName = generateTableName
+      val tablePath = tmp.getCanonicalPath
+
+      spark.sql(
+        s"""
+           |create table $tableName (
+           |  id int,
+           |  name string,
+           |  price double,
+           |  ts long
+           |) using hudi
+           | location '$tablePath'
+           | options (
+           |  primaryKey = 'id',
+           |  preCombineField = 'ts'
+           | )
+       """.stripMargin)
+
+      val metadataTablePath = 
s"$tablePath/${HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH}"
+
+      withSQLConf("spark." + HoodieMetadataConfig.ENABLE.key -> "false") {
+        spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
+      }
+      assertResult(false)(existsPath(metadataTablePath))
+
+      checkAnswer(s"select id, name, price, ts from $tableName")(
+        Seq(1, "a1", 10.0, 1000)
+      )
+    }
+  }
+
   before {
     val testPropsFilePath = new 
File("src/test/resources/external-config").getAbsolutePath
     setEnv(DFSPropertiesConfiguration.CONF_FILE_DIR_ENV_NAME, 
testPropsFilePath)

Reply via email to