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

richox pushed a commit to branch dev-v6.0.0-decimal-cast
in repository https://gitbox.apache.org/repos/asf/auron.git

commit 21db8f750fae4a599caa251c5ee72a6ce6d3bc48
Author: zhangli20 <[email protected]>
AuthorDate: Sat Mar 7 16:15:57 2026 +0800

    supports spark.blaze.enableScanFallback
---
 .../src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala      | 7 +++++--
 .../src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala      | 7 +++++--
 .../org/apache/spark/sql/blaze/BlazeSparkSessionExtension.scala    | 5 +++++
 3 files changed, 15 insertions(+), 4 deletions(-)

diff --git 
a/spark-extension-shims-spark241kwaiae/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala
 
b/spark-extension-shims-spark241kwaiae/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala
index 60b203e6..3db803eb 100644
--- 
a/spark-extension-shims-spark241kwaiae/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala
+++ 
b/spark-extension-shims-spark241kwaiae/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala
@@ -746,7 +746,7 @@ case class BlazeRuleEngine(sparkSession: SparkSession) 
extends Rule[LogicalPlan]
     plan.foreachUp {
       case p @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) =>
         // non parquet table rule
-        if (!(
+        if (sparkSession.conf.getConf(enableScanFallback) && !(
           BlazeConverters.enableScanParquet && 
fsRelation.fileFormat.isInstanceOf[ParquetFileFormat]
             || BlazeConverters.enableScanOrc && 
fsRelation.fileFormat.isInstanceOf[OrcFileFormat]
         )) {
@@ -775,7 +775,9 @@ case class BlazeRuleEngine(sparkSession: SparkSession) 
extends Rule[LogicalPlan]
         }
 
       case h: HiveTableRelation =>
-        turnOffBlazeWithReason(h.conf, BlazeMissPatterns.NonParquetFormat)
+        if (sparkSession.conf.getConf(enableScanFallback)) {
+          turnOffBlazeWithReason(h.conf, BlazeMissPatterns.NonParquetFormat)
+        }
 
       case _ =>
     }
@@ -783,6 +785,7 @@ case class BlazeRuleEngine(sparkSession: SparkSession) 
extends Rule[LogicalPlan]
   }
 
   private def turnOffBlazeWithReason(planConf: SQLConf, blazeMissPattern: 
String): Unit = {
+    logWarning(s"Turn off Blaze due to: $blazeMissPattern")
     planConf.setConf(blazeEnabledKey, false)
     sparkSession.sparkContext.conf
       .set(BlazeRuleEngine.blazeMissPatterns, blazeMissPattern)
diff --git 
a/spark-extension-shims-spark3/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala
 
b/spark-extension-shims-spark3/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala
index 8b74e735..03db82a4 100644
--- 
a/spark-extension-shims-spark3/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala
+++ 
b/spark-extension-shims-spark3/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala
@@ -1063,7 +1063,7 @@ case class BlazeRuleEngine(sparkSession: SparkSession) 
extends Rule[LogicalPlan]
     plan.foreachUp {
       case p @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) =>
         // non parquet table rule
-        if (!(
+        if (sparkSession.conf.getConf(enableScanFallback) && !(
           BlazeConverters.enableScanParquet && 
fsRelation.fileFormat.isInstanceOf[ParquetFileFormat]
             || BlazeConverters.enableScanOrc && 
fsRelation.fileFormat.isInstanceOf[OrcFileFormat]
           )) {
@@ -1092,7 +1092,9 @@ case class BlazeRuleEngine(sparkSession: SparkSession) 
extends Rule[LogicalPlan]
         }
 
       case h: HiveTableRelation =>
-        turnOffBlazeWithReason(h.conf, BlazeMissPatterns.NonParquetFormat)
+        if (sparkSession.conf.getConf(enableScanFallback)) {
+          turnOffBlazeWithReason(h.conf, BlazeMissPatterns.NonParquetFormat)
+        }
 
       case _ =>
     }
@@ -1100,6 +1102,7 @@ case class BlazeRuleEngine(sparkSession: SparkSession) 
extends Rule[LogicalPlan]
   }
 
   private def turnOffBlazeWithReason(planConf: SQLConf, blazeMissPattern: 
String): Unit = {
+    logWarning(s"Turn off Blaze due to: $blazeMissPattern")
     planConf.setConf(blazeEnabledKey, false)
     sparkSession.sparkContext.conf
       .set(BlazeRuleEngine.blazeMissPatterns, blazeMissPattern)
diff --git 
a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeSparkSessionExtension.scala
 
b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeSparkSessionExtension.scala
index e26b9a7f..20ba41b0 100644
--- 
a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeSparkSessionExtension.scala
+++ 
b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeSparkSessionExtension.scala
@@ -49,6 +49,11 @@ object BlazeSparkSessionExtension extends Logging {
     .booleanConf
     .createWithDefault(true)
 
+  lazy val enableScanFallback: ConfigEntry[Boolean] = SQLConf
+    .buildConf("spark.blaze.enableScanFallback")
+    .booleanConf
+    .createWithDefault(false)
+
   def dumpSimpleSparkPlanTreeNode(exec: SparkPlan, depth: Int = 0): Unit = {
     val nodeName = exec.nodeName
     val convertible = exec

Reply via email to