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 a575395a6 [GLUTEN-6877][CH] Support anti/semi join with inequal join 
condition (#6913)
a575395a6 is described below

commit a575395a60d2f2fcb17182cc5a183de07cb601e7
Author: lgbo <[email protected]>
AuthorDate: Fri Aug 23 23:23:28 2024 +0800

    [GLUTEN-6877][CH] Support anti/semi join with inequal join condition (#6913)
    
    * support anti/semi join with mixed join condition
    
    * enable uts
    
    * fix tests
---
 .../execution/CHHashJoinExecTransformer.scala      | 26 ++++--------
 .../apache/gluten/utils/CHJoinValidateUtil.scala   | 48 ++++++++--------------
 ...nClickHouseColumnarMemorySortShuffleSuite.scala |  2 +-
 .../GlutenClickHouseColumnarShuffleAQESuite.scala  |  2 +-
 .../GlutenClickHouseDSV2ColumnarShuffleSuite.scala |  2 +-
 .../execution/GlutenClickHouseDSV2Suite.scala      |  2 +-
 .../execution/GlutenClickHouseDecimalSuite.scala   |  2 +-
 .../GlutenClickHouseTPCDSAbstractSuite.scala       |  6 +--
 ...lickHouseTPCHNullableColumnarShuffleSuite.scala |  2 +-
 .../GlutenClickHouseTPCHNullableSuite.scala        |  2 +-
 .../execution/GlutenClickHouseTPCHSuite.scala      |  2 +-
 ...kHouseTPCDSParquetColumnarShuffleAQESuite.scala |  2 +-
 ...nClickHouseTPCDSParquetGraceHashJoinSuite.scala |  3 +-
 ...nClickHouseTPCDSParquetSortMergeJoinSuite.scala |  5 ++-
 ...ckHouseTPCHColumnarShuffleParquetAQESuite.scala |  2 +-
 ...enClickHouseTPCHParquetAQEConcurrentSuite.scala |  2 -
 .../tpch/GlutenClickHouseTPCHParquetAQESuite.scala |  2 +-
 .../GlutenClickHouseTPCHSaltNullParquetSuite.scala |  2 +-
 cpp-ch/local-engine/Common/QueryContext.cpp        |  2 +-
 cpp-ch/local-engine/Parser/JoinRelParser.cpp       | 13 +++---
 ...ickHouseRSSColumnarMemorySortShuffleSuite.scala |  2 +-
 ...lutenClickHouseRSSColumnarShuffleAQESuite.scala |  2 +-
 22 files changed, 54 insertions(+), 79 deletions(-)

diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala
index 2dd45281e..9b6b2958c 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala
@@ -34,15 +34,12 @@ import com.google.protobuf.{Any, StringValue}
 import io.substrait.proto.JoinRel
 
 object JoinTypeTransform {
-  def toNativeJoinType(joinType: JoinType): JoinType = {
-    joinType match {
-      case ExistenceJoin(_) =>
-        LeftSemi
-      case _ =>
-        joinType
-    }
-  }
 
+  // ExistenceJoin is introduced in #SPARK-14781. It returns all rows from the 
left table with
+  // a new column to indecate whether the row is matched in the right table.
+  // Indeed, the ExistenceJoin is transformed into left any join in CH.
+  // We don't have left any join in substrait, so use left semi join instead.
+  // and isExistenceJoin is set to true to indicate that it is an existence 
join.
   def toSubstraitJoinType(sparkJoin: JoinType, buildRight: Boolean): 
JoinRel.JoinType =
     sparkJoin match {
       case _: InnerLike =>
@@ -104,7 +101,7 @@ case class CHShuffledHashJoinExecTransformer(
   override protected def doValidateInternal(): ValidationResult = {
     val shouldFallback =
       CHJoinValidateUtil.shouldFallback(
-        ShuffleHashJoinStrategy(finalJoinType),
+        ShuffleHashJoinStrategy(joinType),
         left.outputSet,
         right.outputSet,
         condition)
@@ -113,7 +110,6 @@ case class CHShuffledHashJoinExecTransformer(
     }
     super.doValidateInternal()
   }
-  private val finalJoinType = JoinTypeTransform.toNativeJoinType(joinType)
 
   override def genJoinParameters(): Any = {
     val (isBHJ, isNullAwareAntiJoin, buildHashTableId): (Int, Int, String) = 
(0, 0, "")
@@ -226,7 +222,7 @@ case class CHBroadcastHashJoinExecTransformer(
   override protected def doValidateInternal(): ValidationResult = {
     val shouldFallback =
       CHJoinValidateUtil.shouldFallback(
-        BroadcastHashJoinStrategy(finalJoinType),
+        BroadcastHashJoinStrategy(joinType),
         left.outputSet,
         right.outputSet,
         condition)
@@ -255,7 +251,7 @@ case class CHBroadcastHashJoinExecTransformer(
     val context =
       BroadCastHashJoinContext(
         buildKeyExprs,
-        finalJoinType,
+        joinType,
         buildSide == BuildRight,
         isMixedCondition(condition),
         joinType.isInstanceOf[ExistenceJoin],
@@ -278,12 +274,6 @@ case class CHBroadcastHashJoinExecTransformer(
     res
   }
 
-  // ExistenceJoin is introduced in #SPARK-14781. It returns all rows from the 
left table with
-  // a new column to indecate whether the row is matched in the right table.
-  // Indeed, the ExistenceJoin is transformed into left any join in CH.
-  // We don't have left any join in substrait, so use left semi join instead.
-  // and isExistenceJoin is set to true to indicate that it is an existence 
join.
-  private val finalJoinType = JoinTypeTransform.toNativeJoinType(joinType)
   override protected lazy val substraitJoinType: JoinRel.JoinType = {
     JoinTypeTransform.toSubstraitJoinType(joinType, buildSide == BuildRight)
   }
diff --git 
a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala
 
b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala
index 0f5b5e2c4..b86482720 100644
--- 
a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala
+++ 
b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala
@@ -29,13 +29,10 @@ case class BroadcastHashJoinStrategy(joinType: JoinType) 
extends JoinStrategy {}
 case class SortMergeJoinStrategy(joinType: JoinType) extends JoinStrategy {}
 
 /**
- * The logic here is that if it is not an equi-join spark will create BNLJ, 
which will fallback, if
- * it is an equi-join, spark will create BroadcastHashJoin or ShuffleHashJoin, 
for these join types,
- * we need to filter For cases that cannot be handled by the backend, 1 there 
are at least two
- * different tables column and Literal in the condition Or condition for 
comparison, for example: (a
- * join b on a.a1 = b.b1 and (a.a2 > 1 or b.b2 < 2) ) 2 tow join key for 
inequality comparison (!= ,
- * > , <), for example: (a join b on a.a1 > b.b1) There will be a fallback for 
Nullaware Jion For
- * Existence Join which is just an optimization of exist subquery, it will 
also fallback
+ * BroadcastHashJoinStrategy and ShuffleHashJoinStrategy are relatively 
complete, They support
+ * left/right/inner full/anti/semi join, existence Join, and also support join 
contiditions with
+ * columns from both sides. e.g. (a join b on a.a1 = b.b1 and a.a2 > 1 and 
b.b2 < 2)
+ * SortMergeJoinStrategy is not fully supported for all cases in CH.
  */
 
 object CHJoinValidateUtil extends Logging {
@@ -52,33 +49,24 @@ object CHJoinValidateUtil extends Logging {
       leftOutputSet: AttributeSet,
       rightOutputSet: AttributeSet,
       condition: Option[Expression]): Boolean = {
-    var shouldFallback = false
-    val joinType = joinStrategy.joinType
 
-    if (!joinType.isInstanceOf[ExistenceJoin] && 
joinType.sql.contains("INNER")) {
-      shouldFallback = false;
-    } else if (
+    val hasMixedFilterCondition =
       condition.isDefined && hasTwoTableColumn(leftOutputSet, rightOutputSet, 
condition.get)
-    ) {
-      shouldFallback = joinStrategy match {
-        case BroadcastHashJoinStrategy(joinTy) =>
-          joinTy.sql.contains("SEMI") || joinTy.sql.contains("ANTI")
-        case SortMergeJoinStrategy(_) => true
-        case ShuffleHashJoinStrategy(joinTy) =>
-          joinTy.sql.contains("SEMI") || joinTy.sql.contains("ANTI")
-        case UnknownJoinStrategy(joinTy) =>
-          joinTy.sql.contains("SEMI") || joinTy.sql.contains("ANTI")
-      }
-    } else {
-      shouldFallback = joinStrategy match {
-        case SortMergeJoinStrategy(joinTy) =>
-          joinTy.sql.contains("SEMI") || joinTy.sql.contains("ANTI") || 
joinTy.toString.contains(
-            "ExistenceJoin")
-        case _ => false
-      }
+    val shouldFallback = joinStrategy match {
+      case SortMergeJoinStrategy(joinType) =>
+        if (!joinType.isInstanceOf[ExistenceJoin] && 
joinType.sql.contains("INNER")) {
+          false
+        } else {
+          joinType.sql.contains("SEMI") || joinType.sql.contains("ANTI") || 
joinType.toString
+            .contains("ExistenceJoin") || hasMixedFilterCondition
+        }
+      case UnknownJoinStrategy(joinType) =>
+        throw new IllegalArgumentException(s"Unknown join type $joinStrategy")
+      case _ => false
     }
+
     if (shouldFallback) {
-      logError(s"Fallback for join type $joinType")
+      logError(s"Fallback for join type $joinStrategy")
     }
     shouldFallback
   }
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarMemorySortShuffleSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarMemorySortShuffleSuite.scala
index b9d580c72..4c49cc2d9 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarMemorySortShuffleSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarMemorySortShuffleSuite.scala
@@ -119,7 +119,7 @@ class GlutenClickHouseColumnarMemorySortShuffleSuite
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("TPCH Q22") {
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarShuffleAQESuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarShuffleAQESuite.scala
index 10e5c7534..e5da78de3 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarShuffleAQESuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarShuffleAQESuite.scala
@@ -163,7 +163,7 @@ class GlutenClickHouseColumnarShuffleAQESuite
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("TPCH Q22") {
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDSV2ColumnarShuffleSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDSV2ColumnarShuffleSuite.scala
index dd997832d..17bd9912b 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDSV2ColumnarShuffleSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDSV2ColumnarShuffleSuite.scala
@@ -168,7 +168,7 @@ class GlutenClickHouseDSV2ColumnarShuffleSuite extends 
GlutenClickHouseTPCHAbstr
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("TPCH Q22") {
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDSV2Suite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDSV2Suite.scala
index 08393ccfe..a58b6b1c1 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDSV2Suite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDSV2Suite.scala
@@ -126,7 +126,7 @@ class GlutenClickHouseDSV2Suite extends 
GlutenClickHouseTPCHAbstractSuite {
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("TPCH Q22") {
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala
index cf1bdd296..bd831e64b 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala
@@ -343,7 +343,7 @@ class GlutenClickHouseDecimalSuite
               decimalTPCHTables.foreach {
                 dt =>
                   {
-                    val fallBack = (sql_num == 16 || sql_num == 21)
+                    val fallBack = (sql_num == 16)
                     val compareResult = !dt._2.contains(sql_num)
                     val native = if (fallBack) "fallback" else "native"
                     val compare = if (compareResult) "compare" else "noCompare"
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala
index f2a1e5a71..03b26fa98 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala
@@ -62,11 +62,7 @@ abstract class GlutenClickHouseTPCDSAbstractSuite
         })
 
   protected def fallbackSets(isAqe: Boolean): Set[Int] = {
-    val more = if (isSparkVersionGE("3.5")) Set(44, 67, 70) else Set.empty[Int]
-
-    // q16 smj + left semi + not condition
-    // Q94 BroadcastHashJoin, LeftSemi, NOT condition
-    Set(16, 94) | more
+    if (isSparkVersionGE("3.5")) Set(44, 67, 70) else Set.empty[Int]
   }
   protected def excludedTpcdsQueries: Set[String] = Set(
     "q66" // inconsistent results
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableColumnarShuffleSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableColumnarShuffleSuite.scala
index c5f67f45d..5f9aa0dbd 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableColumnarShuffleSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableColumnarShuffleSuite.scala
@@ -171,7 +171,7 @@ class GlutenClickHouseTPCHNullableColumnarShuffleSuite 
extends GlutenClickHouseT
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("TPCH Q22") {
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableSuite.scala
index 7f62c6993..e0e4d3380 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHNullableSuite.scala
@@ -174,7 +174,7 @@ class GlutenClickHouseTPCHNullableSuite extends 
GlutenClickHouseTPCHAbstractSuit
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("TPCH Q22") {
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala
index 1c09449c8..f25a13132 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala
@@ -175,7 +175,7 @@ class GlutenClickHouseTPCHSuite extends 
GlutenClickHouseTPCHAbstractSuite {
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("TPCH Q22") {
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala
index 1fd8983f5..3e1507bf1 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala
@@ -239,6 +239,6 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite
         | LIMIT 100 ;
         |""".stripMargin
     // There are some BroadcastHashJoin with NOT condition
-    compareResultsAgainstVanillaSpark(sql, true, { df => }, false)
+    compareResultsAgainstVanillaSpark(sql, true, { df => })
   }
 }
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala
index 716ea5761..4b4a0d34e 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala
@@ -31,9 +31,10 @@ class GlutenClickHouseTPCDSParquetGraceHashJoinSuite extends 
GlutenClickHouseTPC
       .set("spark.io.compression.codec", "snappy")
       .set("spark.sql.shuffle.partitions", "5")
       .set("spark.sql.autoBroadcastJoinThreshold", "10MB")
-      .set("spark.memory.offHeap.size", "8g")
+      .set("spark.memory.offHeap.size", "6g")
       
.set("spark.gluten.sql.columnar.backend.ch.runtime_settings.join_algorithm", 
"grace_hash")
       
.set("spark.gluten.sql.columnar.backend.ch.runtime_settings.max_bytes_in_join", 
"314572800")
+      .setMaster("local[2]")
   }
 
   executeTPCDSTest(false)
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala
index 509c83054..41436da70 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala
@@ -49,9 +49,10 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends 
GlutenClickHouseTPC
       .set("spark.shuffle.manager", "sort")
       .set("spark.io.compression.codec", "snappy")
       .set("spark.sql.shuffle.partitions", "5")
-      .set("spark.sql.autoBroadcastJoinThreshold", "10MB")
-      .set("spark.memory.offHeap.size", "8g")
+      .set("spark.sql.autoBroadcastJoinThreshold", "-1")
+      .set("spark.memory.offHeap.size", "6g")
       .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false")
+      .setMaster("local[2]")
   }
 
   executeTPCDSTest(false)
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala
index c2e2f9f55..39dc7baf9 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala
@@ -275,7 +275,7 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) {
+    runTPCHQuery(21) {
       df =>
         val plans = collect(df.queryExecution.executedPlan) {
           case scanExec: BasicScanExecTransformer => scanExec
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala
index 8c706f683..e1dd33912 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala
@@ -45,7 +45,6 @@ class GlutenClickHouseTPCHParquetAQEConcurrentSuite
       .set("spark.shuffle.manager", "sort")
       .set("spark.io.compression.codec", "snappy")
       .set("spark.sql.shuffle.partitions", "5")
-      .set("spark.sql.autoBroadcastJoinThreshold", "10MB")
       .set("spark.sql.adaptive.enabled", "true")
       .set("spark.sql.autoBroadcastJoinThreshold", "-1")
   }
@@ -82,5 +81,4 @@ class GlutenClickHouseTPCHParquetAQEConcurrentSuite
     queries.map(queryId => runTPCHQuery(queryId) { df => })
 
   }
-
 }
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala
index 1d8389b48..2aadac05d 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala
@@ -209,7 +209,7 @@ class GlutenClickHouseTPCHParquetAQESuite
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("TPCH Q22") {
diff --git 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala
 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala
index 0efc1414c..e21df203d 100644
--- 
a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala
@@ -335,7 +335,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends 
GlutenClickHouseTPCHAbstr
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("GLUTEN-2115: Fix wrong number of records shuffle written") {
diff --git a/cpp-ch/local-engine/Common/QueryContext.cpp 
b/cpp-ch/local-engine/Common/QueryContext.cpp
index ff9c15115..0abff2fc1 100644
--- a/cpp-ch/local-engine/Common/QueryContext.cpp
+++ b/cpp-ch/local-engine/Common/QueryContext.cpp
@@ -172,4 +172,4 @@ double currentThreadGroupMemoryUsageRatio()
     }
     return 
static_cast<double>(CurrentThread::getGroup()->memory_tracker.get()) / 
CurrentThread::getGroup()->memory_tracker.getSoftLimit();
 }
-}
\ No newline at end of file
+}
diff --git a/cpp-ch/local-engine/Parser/JoinRelParser.cpp 
b/cpp-ch/local-engine/Parser/JoinRelParser.cpp
index 0446a397c..2959e1986 100644
--- a/cpp-ch/local-engine/Parser/JoinRelParser.cpp
+++ b/cpp-ch/local-engine/Parser/JoinRelParser.cpp
@@ -681,14 +681,14 @@ bool JoinRelParser::couldRewriteToMultiJoinOnClauses(
                 auto optional_keys = parse_join_keys(current_expr);
                 if (!optional_keys)
                 {
-                    LOG_ERROR(getLogger("JoinRelParser"), "Not equal 
comparison for keys from both tables");
+                    LOG_DEBUG(getLogger("JoinRelParser"), "Not equal 
comparison for keys from both tables");
                     return false;
                 }
                 join_on_clause.addKey(optional_keys->first, 
optional_keys->second, false);
             }
             else
             {
-                LOG_ERROR(getLogger("JoinRelParser"), "And or equals function 
is expected");
+                LOG_DEBUG(getLogger("JoinRelParser"), "And or equals function 
is expected");
                 return false;
             }
         }
@@ -701,7 +701,8 @@ bool JoinRelParser::couldRewriteToMultiJoinOnClauses(
         expression_stack.pop_back();
         if (!check_function("or", current_expr))
         {
-            LOG_ERROR(getLogger("JoinRelParser"), "Not an or expression");
+            LOG_DEBUG(getLogger("JoinRelParser"), "Not an or expression");
+            return false;
         }
 
         auto get_current_join_on_clause = [&]()
@@ -719,7 +720,7 @@ bool JoinRelParser::couldRewriteToMultiJoinOnClauses(
                 auto optional_keys = parse_join_keys(arg.value());
                 if (!optional_keys)
                 {
-                    LOG_ERROR(getLogger("JoinRelParser"), "Not equal 
comparison for keys from both tables");
+                    LOG_DEBUG(getLogger("JoinRelParser"), "Not equal 
comparison for keys from both tables");
                     return false;
                 }
                 get_current_join_on_clause()->addKey(optional_keys->first, 
optional_keys->second, false);
@@ -728,7 +729,7 @@ bool JoinRelParser::couldRewriteToMultiJoinOnClauses(
             {
                 if (!parse_and_expression(arg.value(), 
*get_current_join_on_clause()))
                 {
-                    LOG_ERROR(getLogger("JoinRelParser"), "Parse and 
expression failed");
+                    LOG_DEBUG(getLogger("JoinRelParser"), "Parse and 
expression failed");
                     return false;
                 }
             }
@@ -738,7 +739,7 @@ bool JoinRelParser::couldRewriteToMultiJoinOnClauses(
             }
             else
             {
-                LOG_ERROR(getLogger("JoinRelParser"), "Unknow function");
+                LOG_DEBUG(getLogger("JoinRelParser"), "Unknow function");
                 return false;
             }
         }
diff --git 
a/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarMemorySortShuffleSuite.scala
 
b/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarMemorySortShuffleSuite.scala
index ddef1d87c..ef1c4180b 100644
--- 
a/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarMemorySortShuffleSuite.scala
+++ 
b/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarMemorySortShuffleSuite.scala
@@ -127,7 +127,7 @@ class GlutenClickHouseRSSColumnarMemorySortShuffleSuite
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("TPCH Q22") {
diff --git 
a/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarShuffleAQESuite.scala
 
b/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarShuffleAQESuite.scala
index c3d71ba53..50220264e 100644
--- 
a/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarShuffleAQESuite.scala
+++ 
b/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarShuffleAQESuite.scala
@@ -162,7 +162,7 @@ class GlutenClickHouseRSSColumnarShuffleAQESuite
   }
 
   test("TPCH Q21") {
-    runTPCHQuery(21, noFallBack = false) { df => }
+    runTPCHQuery(21) { df => }
   }
 
   test("TPCH Q22") {


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

Reply via email to