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

changchen pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
     new 9ac7bf57bb Revert "Add RichSparkConf to simplify the interoperations 
with gluten config entries (#9876)" (#9905)
9ac7bf57bb is described below

commit 9ac7bf57bb3b4f55e30c988de237818997f1a25c
Author: Chang chen <[email protected]>
AuthorDate: Sat Jun 7 13:55:28 2025 +0800

    Revert "Add RichSparkConf to simplify the interoperations with gluten 
config entries (#9876)" (#9905)
    
    This reverts commit fa4fa941b3a538d318a28775e40d81c670994a8f.
    
    Co-authored-by: Chang chen <[email protected]>
---
 .../backendsapi/velox/VeloxListenerApi.scala       | 29 ++++++-----
 .../apache/gluten/utils/SharedLibraryLoader.scala  |  9 ++--
 .../scala/org/apache/gluten/GlutenPlugin.scala     | 56 ++++++++++++++--------
 .../spark/sql/internal/SparkConfigUtil.scala       | 37 +-------------
 .../org/apache/gluten/expression/UDFMappings.scala |  7 ++-
 5 files changed, 64 insertions(+), 74 deletions(-)

diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala
index 44bfd0aab3..b9e24e5b9c 100644
--- 
a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala
@@ -43,7 +43,6 @@ import 
org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules
 import 
org.apache.spark.sql.execution.datasources.velox.{VeloxParquetWriterInjects, 
VeloxRowSplitter}
 import org.apache.spark.sql.expression.UDFResolver
 import org.apache.spark.sql.internal.{GlutenConfigUtil, StaticSQLConf}
-import org.apache.spark.sql.internal.SparkConfigUtil._
 import org.apache.spark.util.{SparkDirectoryUtil, SparkResourceUtil, 
SparkShutdownManagerUtil}
 
 import org.apache.commons.lang3.StringUtils
@@ -60,15 +59,18 @@ class VeloxListenerApi extends ListenerApi with Logging {
     // When the Velox cache is enabled, the Velox file handle cache should 
also be enabled.
     // Otherwise, a 'reference id not found' error may occur.
     if (
-      conf.get(COLUMNAR_VELOX_CACHE_ENABLED) &&
-      !conf.get(COLUMNAR_VELOX_FILE_HANDLE_CACHE_ENABLED)
+      conf.getBoolean(COLUMNAR_VELOX_CACHE_ENABLED.key, false) &&
+      !conf.getBoolean(COLUMNAR_VELOX_FILE_HANDLE_CACHE_ENABLED.key, false)
     ) {
       throw new IllegalArgumentException(
         s"${COLUMNAR_VELOX_CACHE_ENABLED.key} and " +
           s"${COLUMNAR_VELOX_FILE_HANDLE_CACHE_ENABLED.key} should be enabled 
together.")
     }
 
-    if (conf.get(COLUMNAR_VELOX_CACHE_ENABLED) && conf.get(LOAD_QUANTUM) > 8 * 
1024 * 1024) {
+    if (
+      conf.getBoolean(COLUMNAR_VELOX_CACHE_ENABLED.key, false) &&
+      conf.getSizeAsBytes(LOAD_QUANTUM.key, LOAD_QUANTUM.defaultValueString) > 
8 * 1024 * 1024
+    ) {
       throw new IllegalArgumentException(
         s"Velox currently only support up to 8MB load quantum size " +
           s"on SSD cache enabled by ${COLUMNAR_VELOX_CACHE_ENABLED.key}, " +
@@ -98,11 +100,13 @@ class VeloxListenerApi extends ListenerApi with Logging {
           s" the recommended size 
${ByteUnit.BYTE.toMiB(desiredOverheadSize)}MiB." +
           s" This may cause OOM.")
     }
-    conf.set(GlutenConfig.COLUMNAR_OVERHEAD_SIZE_IN_BYTES, overheadSize)
+    conf.set(GlutenConfig.COLUMNAR_OVERHEAD_SIZE_IN_BYTES.key, 
overheadSize.toString)
 
     // Sql table cache serializer.
-    if (conf.get(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED)) {
-      conf.set(StaticSQLConf.SPARK_CACHE_SERIALIZER, 
classOf[ColumnarCachedBatchSerializer].getName)
+    if (conf.getBoolean(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, 
defaultValue = false)) {
+      conf.set(
+        StaticSQLConf.SPARK_CACHE_SERIALIZER.key,
+        classOf[ColumnarCachedBatchSerializer].getName)
     }
 
     // Static initializers for driver.
@@ -150,8 +154,8 @@ class VeloxListenerApi extends ListenerApi with Logging {
   private def initialize(conf: SparkConf, isDriver: Boolean): Unit = {
     // Sets this configuration only once, since not undoable.
     // DebugInstance should be created first.
-    if (conf.get(GlutenConfig.DEBUG_KEEP_JNI_WORKSPACE)) {
-      val debugDir = conf.get(GlutenConfig.DEBUG_KEEP_JNI_WORKSPACE_DIR)
+    if (conf.getBoolean(GlutenConfig.DEBUG_KEEP_JNI_WORKSPACE.key, 
defaultValue = false)) {
+      val debugDir = conf.get(GlutenConfig.DEBUG_KEEP_JNI_WORKSPACE_DIR.key)
       JniWorkspace.enableDebug(debugDir)
     } else {
       JniWorkspace.initializeDefault(
@@ -198,11 +202,11 @@ class VeloxListenerApi extends ListenerApi with Logging {
     SharedLibraryLoader.load(conf, loader)
 
     // Load backend libraries.
-    val libPath = conf.get(GlutenConfig.GLUTEN_LIB_PATH)
+    val libPath = conf.get(GlutenConfig.GLUTEN_LIB_PATH.key, StringUtils.EMPTY)
     if (StringUtils.isNotBlank(libPath)) { // Path based load. Ignore all 
other loadees.
       JniLibLoader.loadFromPath(libPath)
     } else {
-      val baseLibName = conf.get(GlutenConfig.GLUTEN_LIB_NAME)
+      val baseLibName = conf.get(GlutenConfig.GLUTEN_LIB_NAME.key, "gluten")
       loader.load(s"$platformLibDir/${System.mapLibraryName(baseLibName)}")
       
loader.load(s"$platformLibDir/${System.mapLibraryName(VeloxBackend.BACKEND_NAME)}")
     }
@@ -220,7 +224,8 @@ class VeloxListenerApi extends ListenerApi with Logging {
   }
 
   private def addIfNeedMemoryDumpShutdownHook(conf: SparkConf): Unit = {
-    val memoryDumpOnExit = conf.get(MEMORY_DUMP_ON_EXIT)
+    val memoryDumpOnExit =
+      conf.get(MEMORY_DUMP_ON_EXIT.key, 
MEMORY_DUMP_ON_EXIT.defaultValueString).toBoolean
     if (memoryDumpOnExit) {
       SparkShutdownManagerUtil.addHook(
         () => {
diff --git 
a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoader.scala
 
b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoader.scala
index 109f3f014a..3632202de9 100755
--- 
a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoader.scala
+++ 
b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoader.scala
@@ -21,7 +21,6 @@ import org.apache.gluten.exception.GlutenException
 import org.apache.gluten.jni.JniLibLoader
 
 import org.apache.spark.SparkConf
-import org.apache.spark.sql.internal.SparkConfigUtil._
 
 import scala.sys.process._
 
@@ -31,7 +30,9 @@ trait SharedLibraryLoader {
 
 object SharedLibraryLoader {
   def load(conf: SparkConf, jni: JniLibLoader): Unit = {
-    val shouldLoad = conf.get(GlutenConfig.GLUTEN_LOAD_LIB_FROM_JAR)
+    val shouldLoad = conf.getBoolean(
+      GlutenConfig.GLUTEN_LOAD_LIB_FROM_JAR.key,
+      GlutenConfig.GLUTEN_LOAD_LIB_FROM_JAR.defaultValue.get)
     if (!shouldLoad) {
       return
     }
@@ -53,9 +54,9 @@ object SharedLibraryLoader {
   }
 
   private def find(conf: SparkConf): SharedLibraryLoader = {
-    val systemName = conf.get(GlutenConfig.GLUTEN_LOAD_LIB_OS)
+    val systemName = conf.getOption(GlutenConfig.GLUTEN_LOAD_LIB_OS.key)
     val loader = if (systemName.isDefined) {
-      val systemVersion = conf.get(GlutenConfig.GLUTEN_LOAD_LIB_OS_VERSION)
+      val systemVersion = 
conf.getOption(GlutenConfig.GLUTEN_LOAD_LIB_OS_VERSION.key)
       if (systemVersion.isEmpty) {
         throw new GlutenException(
           s"${GlutenConfig.GLUTEN_LOAD_LIB_OS_VERSION.key} must be specified 
when specifies the " +
diff --git a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala 
b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala
index 95bebac5ff..611e4e398c 100644
--- a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala
+++ b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala
@@ -33,8 +33,7 @@ import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.softaffinity.SoftAffinityListener
 import org.apache.spark.sql.execution.adaptive.GlutenCostEvaluator
 import org.apache.spark.sql.execution.ui.{GlutenSQLAppStatusListener, 
GlutenUIUtils}
-import org.apache.spark.sql.internal.SparkConfigUtil._
-import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.{SparkConfigUtil, SQLConf}
 import org.apache.spark.sql.internal.StaticSQLConf.SPARK_SESSION_EXTENSIONS
 import org.apache.spark.task.TaskResources
 import org.apache.spark.util.SparkResourceUtil
@@ -63,7 +62,11 @@ private[gluten] class GlutenDriverPlugin extends 
DriverPlugin with Logging {
 
     // Register Gluten listeners
     GlutenSQLAppStatusListener.register(sc)
-    if (conf.get(GLUTEN_SOFT_AFFINITY_ENABLED)) {
+    if (
+      conf.getBoolean(
+        GLUTEN_SOFT_AFFINITY_ENABLED.key,
+        GLUTEN_SOFT_AFFINITY_ENABLED.defaultValue.get)
+    ) {
       SoftAffinityListener.register(sc)
     }
 
@@ -134,13 +137,19 @@ private[gluten] class GlutenDriverPlugin extends 
DriverPlugin with Logging {
   }
 
   private def checkOffHeapSettings(conf: SparkConf): Unit = {
-    if (conf.get(DYNAMIC_OFFHEAP_SIZING_ENABLED)) {
+    if (
+      conf.getBoolean(
+        DYNAMIC_OFFHEAP_SIZING_ENABLED.key,
+        DYNAMIC_OFFHEAP_SIZING_ENABLED.defaultValue.get)
+    ) {
       // When dynamic off-heap sizing is enabled, off-heap mode is not 
strictly required to be
       // enabled. Skip the check.
       return
     }
 
-    if (conf.get(COLUMNAR_MEMORY_UNTRACKED)) {
+    if (
+      conf.getBoolean(COLUMNAR_MEMORY_UNTRACKED.key, 
COLUMNAR_MEMORY_UNTRACKED.defaultValue.get)
+    ) {
       // When untracked memory mode is enabled, off-heap mode is not strictly 
required to be
       // enabled. Skip the check.
       return
@@ -160,19 +169,22 @@ private[gluten] class GlutenDriverPlugin extends 
DriverPlugin with Logging {
 
   private def setPredefinedConfigs(conf: SparkConf): Unit = {
     // Spark SQL extensions
-    val extensionSeq = conf.get(SPARK_SESSION_EXTENSIONS).getOrElse(Seq.empty)
+    val extensionSeq =
+      SparkConfigUtil.getEntryValue(conf, 
SPARK_SESSION_EXTENSIONS).getOrElse(Seq.empty)
     if 
(!extensionSeq.toSet.contains(GlutenSessionExtensions.GLUTEN_SESSION_EXTENSION_NAME))
 {
       conf.set(
-        SPARK_SESSION_EXTENSIONS,
-        Some(extensionSeq :+ 
GlutenSessionExtensions.GLUTEN_SESSION_EXTENSION_NAME))
+        SPARK_SESSION_EXTENSIONS.key,
+        (extensionSeq :+ 
GlutenSessionExtensions.GLUTEN_SESSION_EXTENSION_NAME).mkString(","))
     }
 
     // adaptive custom cost evaluator class
-    val enableGlutenCostEvaluator = 
conf.get(GlutenConfig.COST_EVALUATOR_ENABLED)
+    val enableGlutenCostEvaluator = conf.getBoolean(
+      GlutenConfig.COST_EVALUATOR_ENABLED.key,
+      GlutenConfig.COST_EVALUATOR_ENABLED.defaultValue.get)
     if (enableGlutenCostEvaluator) {
       conf.set(
-        SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS,
-        Some(classOf[GlutenCostEvaluator].getName))
+        SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key,
+        classOf[GlutenCostEvaluator].getName)
     }
 
     // check memory off-heap enabled and size.
@@ -182,33 +194,39 @@ private[gluten] class GlutenDriverPlugin extends 
DriverPlugin with Logging {
     val offHeapSize = conf.getSizeAsBytes(SPARK_OFFHEAP_SIZE_KEY)
 
     // Set off-heap size in bytes.
-    conf.set(COLUMNAR_OFFHEAP_SIZE_IN_BYTES, offHeapSize)
+    conf.set(COLUMNAR_OFFHEAP_SIZE_IN_BYTES.key, offHeapSize.toString)
 
     // Set off-heap size in bytes per task.
     val taskSlots = SparkResourceUtil.getTaskSlots(conf)
-    conf.set(NUM_TASK_SLOTS_PER_EXECUTOR, taskSlots)
+    conf.set(NUM_TASK_SLOTS_PER_EXECUTOR.key, taskSlots.toString)
     val offHeapPerTask = offHeapSize / taskSlots
-    conf.set(COLUMNAR_TASK_OFFHEAP_SIZE_IN_BYTES, offHeapPerTask)
+    conf.set(COLUMNAR_TASK_OFFHEAP_SIZE_IN_BYTES.key, offHeapPerTask.toString)
 
     // Pessimistic off-heap sizes, with the assumption that all non-borrowable 
storage memory
     // determined by spark.memory.storageFraction was used.
     val fraction = 1.0d - conf.getDouble("spark.memory.storageFraction", 0.5d)
     val conservativeOffHeapPerTask = (offHeapSize * fraction).toLong / 
taskSlots
-    conf.set(COLUMNAR_CONSERVATIVE_TASK_OFFHEAP_SIZE_IN_BYTES, 
conservativeOffHeapPerTask)
+    conf.set(
+      COLUMNAR_CONSERVATIVE_TASK_OFFHEAP_SIZE_IN_BYTES.key,
+      conservativeOffHeapPerTask.toString)
 
     // Disable vanilla columnar readers, to prevent columnar-to-columnar 
conversions.
     // FIXME: Do we still need this trick since
     //  https://github.com/apache/incubator-gluten/pull/1931 was merged?
-    if (!conf.get(VANILLA_VECTORIZED_READERS_ENABLED)) {
+    if (
+      !conf.getBoolean(
+        VANILLA_VECTORIZED_READERS_ENABLED.key,
+        VANILLA_VECTORIZED_READERS_ENABLED.defaultValue.get)
+    ) {
       // FIXME Hongze 22/12/06
       //  BatchScan.scala in shim was not always loaded by class loader.
       //  The file should be removed and the "ClassCastException" issue caused 
by
       //  spark.sql.<format>.enableVectorizedReader=true should be fixed in 
another way.
       //  Before the issue is fixed we force the use of vanilla row reader by 
using
       //  the following statement.
-      conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED, false)
-      conf.set(SQLConf.ORC_VECTORIZED_READER_ENABLED, false)
-      conf.set(SQLConf.CACHE_VECTORIZED_READER_ENABLED, false)
+      conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "false")
+      conf.set(SQLConf.ORC_VECTORIZED_READER_ENABLED.key, "false")
+      conf.set(SQLConf.CACHE_VECTORIZED_READER_ENABLED.key, "false")
     }
   }
 }
diff --git 
a/gluten-core/src/main/scala/org/apache/spark/sql/internal/SparkConfigUtil.scala
 
b/gluten-core/src/main/scala/org/apache/spark/sql/internal/SparkConfigUtil.scala
index 9b05aa75d6..945174073e 100644
--- 
a/gluten-core/src/main/scala/org/apache/spark/sql/internal/SparkConfigUtil.scala
+++ 
b/gluten-core/src/main/scala/org/apache/spark/sql/internal/SparkConfigUtil.scala
@@ -16,44 +16,11 @@
  */
 package org.apache.spark.sql.internal
 
-import org.apache.gluten.config.ConfigEntry
-
 import org.apache.spark.SparkConf
-import org.apache.spark.internal.config.{ConfigEntry => SparkConfigEntry}
+import org.apache.spark.internal.config.ConfigEntry
 
 object SparkConfigUtil {
-
-  implicit class RichSparkConf(val conf: SparkConf) {
-    def get[T](entry: SparkConfigEntry[T]): T = {
-      SparkConfigUtil.get(conf, entry)
-    }
-
-    def get[T](entry: ConfigEntry[T]): T = {
-      SparkConfigUtil.get(conf, entry)
-    }
-
-    def set[T](entry: SparkConfigEntry[T], value: T): SparkConf = {
-      SparkConfigUtil.set(conf, entry, value)
-    }
-
-    def set[T](entry: ConfigEntry[T], value: T): SparkConf = {
-      SparkConfigUtil.set(conf, entry, value)
-    }
-  }
-
-  def get[T](conf: SparkConf, entry: SparkConfigEntry[T]): T = {
+  def getEntryValue[T](conf: SparkConf, entry: ConfigEntry[T]): T = {
     conf.get(entry)
   }
-
-  def get[T](conf: SparkConf, entry: ConfigEntry[T]): T = {
-    entry.valueConverter(conf.get(entry.key, entry.defaultValueString))
-  }
-
-  def set[T](conf: SparkConf, entry: SparkConfigEntry[T], value: T): SparkConf 
= {
-    conf.set(entry, value)
-  }
-
-  def set[T](conf: SparkConf, entry: ConfigEntry[T], value: T): SparkConf = {
-    conf.set(entry.key, entry.stringConverter(value))
-  }
 }
diff --git 
a/gluten-substrait/src/main/scala/org/apache/gluten/expression/UDFMappings.scala
 
b/gluten-substrait/src/main/scala/org/apache/gluten/expression/UDFMappings.scala
index baaa6b2d64..568688c0bd 100644
--- 
a/gluten-substrait/src/main/scala/org/apache/gluten/expression/UDFMappings.scala
+++ 
b/gluten-substrait/src/main/scala/org/apache/gluten/expression/UDFMappings.scala
@@ -20,7 +20,6 @@ import org.apache.gluten.config.GlutenConfig
 
 import org.apache.spark.SparkConf
 import org.apache.spark.internal.Logging
-import org.apache.spark.sql.internal.SparkConfigUtil._
 
 import org.apache.commons.lang3.StringUtils
 
@@ -59,19 +58,19 @@ object UDFMappings extends Logging {
   }
 
   def loadFromSparkConf(conf: SparkConf): Unit = {
-    val strHiveUDFs = conf.get(GlutenConfig.GLUTEN_SUPPORTED_HIVE_UDFS)
+    val strHiveUDFs = conf.get(GlutenConfig.GLUTEN_SUPPORTED_HIVE_UDFS.key, "")
     if (!StringUtils.isBlank(strHiveUDFs)) {
       parseStringToMap(strHiveUDFs, hiveUDFMap)
       logDebug(s"loaded hive udf mappings:${hiveUDFMap.toString}")
     }
 
-    val strPythonUDFs = conf.get(GlutenConfig.GLUTEN_SUPPORTED_PYTHON_UDFS)
+    val strPythonUDFs = 
conf.get(GlutenConfig.GLUTEN_SUPPORTED_PYTHON_UDFS.key, "")
     if (!StringUtils.isBlank(strPythonUDFs)) {
       parseStringToMap(strPythonUDFs, pythonUDFMap)
       logDebug(s"loaded python udf mappings:${pythonUDFMap.toString}")
     }
 
-    val strScalaUDFs = conf.get(GlutenConfig.GLUTEN_SUPPORTED_SCALA_UDFS)
+    val strScalaUDFs = conf.get(GlutenConfig.GLUTEN_SUPPORTED_SCALA_UDFS.key, 
"")
     if (!StringUtils.isBlank(strScalaUDFs)) {
       parseStringToMap(strScalaUDFs, scalaUDFMap)
       logDebug(s"loaded scala udf mappings:${scalaUDFMap.toString}")


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

Reply via email to