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

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


The following commit(s) were added to refs/heads/master by this push:
     new c8eb43d60d8 Revert "[opt](nereids) remove redundant hash join 
condition (#48363)" (#49628)
c8eb43d60d8 is described below

commit c8eb43d60d8369d3a7d06de545717da289fc3a71
Author: minghong <zhoumingh...@selectdb.com>
AuthorDate: Tue Apr 1 09:44:39 2025 +0800

    Revert "[opt](nereids) remove redundant hash join condition (#48363)" 
(#49628)
    
    This reverts commit 77a2b22ba6e3e41864bdf87d17abdb071f24b726.
---
 .../java/org/apache/doris/mtmv/MTMVPlanUtil.java   |   1 -
 .../doris/nereids/jobs/executor/Rewriter.java      |   8 +-
 .../org/apache/doris/nereids/rules/RuleType.java   |   1 -
 .../exploration/mv/MaterializedViewUtils.java      |   3 +-
 .../rewrite/EliminateConstHashJoinCondition.java   |  91 ---------------------
 .../nereids/trees/plans/logical/LogicalJoin.java   |   7 --
 .../joinorder/hypergraph/PullupExpressionTest.java |  10 +--
 .../exploration/mv/MaterializedViewUtilsTest.java  |   2 +-
 .../EliminateConstHashJoinConditionTest.java       |  58 -------------
 .../nereids_p0/runtime_filter/runtime_filter.out   | Bin 742 -> 868 bytes
 .../extend_infer_equal_predicate.out               | Bin 30195 -> 30317 bytes
 .../infer_predicate/pull_up_predicate_literal.out  | Bin 78088 -> 82024 bytes
 .../infer_predicate/pull_up_predicate_set_op.out   | Bin 19652 -> 19703 bytes
 .../data/nereids_rules_p0/pkfk/eliminate_inner.out | Bin 6965 -> 7333 bytes
 .../nereids_p0/cte/test_cte_filter_pushdown.groovy |   2 -
 .../cte/test_cte_filter_pushdown.groovy            |   2 +-
 .../filter_push_down/push_filter_through.groovy    |   2 +-
 .../pull_up_predicate_literal.groovy               |   2 +-
 .../agg_optimize_when_uniform.groovy               |  10 +--
 .../agg_with_roll_up/aggregate_with_roll_up.groovy |   1 -
 .../aggregate_without_roll_up.groovy               |   3 +-
 .../mv/dimension/dimension_1.groovy                |   1 -
 .../mv/dimension/dimension_2_full_join.groovy      |   1 -
 .../mv/dimension/dimension_2_inner_join.groovy     |   1 -
 .../mv/dimension/dimension_2_left_join.groovy      |   1 -
 .../mv/dimension/dimension_2_right_join.groovy     |   1 -
 .../mv/join/dphyp_inner/inner_join_dphyp.groovy    |   1 -
 .../mv/join/dphyp_outer/outer_join_dphyp.groovy    |   2 -
 .../mv/join/inner/inner_join.groovy                |   1 -
 .../mv/join/left_outer/outer_join.groovy           |   1 -
 .../inner_join_infer_and_derive.groovy             |   1 -
 .../mv/nested/nested_materialized_view.groovy      |   1 -
 .../predicate_infer/infer_predicate.groovy         |   2 +-
 33 files changed, 15 insertions(+), 202 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java 
b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java
index f6face9c3c9..13a58d82901 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java
@@ -78,7 +78,6 @@ public class MTMVPlanUtil {
         ctx.getSessionVariable().setDisableNereidsRules(
                 String.join(",", ImmutableSet.of(
                         "COMPRESSED_MATERIALIZE_AGG", 
"COMPRESSED_MATERIALIZE_SORT",
-                        "ELIMINATE_CONST_JOIN_CONDITION",
                         RuleType.ADD_DEFAULT_LIMIT.name())));
         ctx.setStartTime();
         if (parentContext != null) {
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java
index ede35d2909e..80c8760d0e5 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java
@@ -60,7 +60,6 @@ import 
org.apache.doris.nereids.rules.rewrite.DeferMaterializeTopNResult;
 import org.apache.doris.nereids.rules.rewrite.EliminateAggCaseWhen;
 import org.apache.doris.nereids.rules.rewrite.EliminateAggregate;
 import org.apache.doris.nereids.rules.rewrite.EliminateAssertNumRows;
-import org.apache.doris.nereids.rules.rewrite.EliminateConstHashJoinCondition;
 import org.apache.doris.nereids.rules.rewrite.EliminateDedupJoinCondition;
 import org.apache.doris.nereids.rules.rewrite.EliminateEmptyRelation;
 import org.apache.doris.nereids.rules.rewrite.EliminateFilter;
@@ -489,14 +488,11 @@ public class Rewriter extends AbstractBatchJobExecutor {
                         custom(RuleType.ADD_PROJECT_FOR_JOIN, 
AddProjectForJoin::new),
                         topDown(new MergeProjects())
                 ),
-                topic("adjust topN project",
+                topic("Adjust topN project",
                         topDown(new MergeProjects(),
                                 new PullUpProjectBetweenTopNAndAgg())),
-                topic("remove const hash join condition",
-                        topDown(new EliminateConstHashJoinCondition())),
-
                 // this rule batch must keep at the end of rewrite to do some 
plan check
-                topic("final rewrite and check",
+                topic("Final rewrite and check",
                         custom(RuleType.CHECK_DATA_TYPES, CheckDataTypes::new),
                         topDown(new PushDownFilterThroughProject(), new 
MergeProjects()),
                         custom(RuleType.ADJUST_CONJUNCTS_RETURN_TYPE, 
AdjustConjunctsReturnType::new),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java
index b8f49473bc2..9cee3077f13 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java
@@ -258,7 +258,6 @@ public enum RuleType {
     ELIMINATE_FILTER(RuleTypeClass.REWRITE),
     ELIMINATE_JOIN(RuleTypeClass.REWRITE),
     ELIMINATE_JOIN_BY_FOREIGN_KEY(RuleTypeClass.REWRITE),
-    ELIMINATE_CONST_JOIN_CONDITION(RuleTypeClass.REWRITE),
     ELIMINATE_JOIN_CONDITION(RuleTypeClass.REWRITE),
     ELIMINATE_FILTER_ON_ONE_RELATION(RuleTypeClass.REWRITE),
     ELIMINATE_SEMI_JOIN(RuleTypeClass.REWRITE),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java
index e92e9628a32..74cc21e0342 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java
@@ -262,8 +262,7 @@ public class MaterializedViewUtils {
                 .getSessionVariable()
                 .getDisableNereidsRuleNames();
         
rewrittenPlanContext.getStatementContext().getConnectContext().getSessionVariable()
-                .setDisableNereidsRules(String.join(",", 
ImmutableSet.of(RuleType.ADD_DEFAULT_LIMIT.name(),
-                        RuleType.ELIMINATE_CONST_JOIN_CONDITION.name())));
+                .setDisableNereidsRules(String.join(",", 
ImmutableSet.of(RuleType.ADD_DEFAULT_LIMIT.name())));
         
rewrittenPlanContext.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES);
         try {
             rewrittenPlanContext.getConnectContext().setSkipAuth(true);
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateConstHashJoinCondition.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateConstHashJoinCondition.java
deleted file mode 100644
index 72e7b462ed6..00000000000
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateConstHashJoinCondition.java
+++ /dev/null
@@ -1,91 +0,0 @@
-// 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.doris.nereids.rules.rewrite;
-
-import org.apache.doris.nereids.rules.Rule;
-import org.apache.doris.nereids.rules.RuleType;
-import org.apache.doris.nereids.trees.expressions.EqualTo;
-import org.apache.doris.nereids.trees.expressions.Expression;
-import org.apache.doris.nereids.trees.expressions.Slot;
-import org.apache.doris.nereids.trees.expressions.SlotReference;
-import org.apache.doris.nereids.trees.plans.Plan;
-import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
-import org.apache.doris.nereids.util.JoinUtils;
-
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Optional;
-
-/**
- * T1 join T2 on T1.a=T2.a and T1.a=1 and T2.a=1
- * T1.a = T2.a can be eliminated
- */
-public class EliminateConstHashJoinCondition extends OneRewriteRuleFactory {
-    public static final Logger LOG = 
LogManager.getLogger(EliminateConstHashJoinCondition.class);
-
-    @Override
-    public Rule build() {
-        return logicalJoin()
-                .when(join -> join.getJoinType().isInnerJoin() || 
join.getJoinType().isSemiJoin())
-                .whenNot(join -> join.isMarkJoin())
-                
.then(EliminateConstHashJoinCondition::eliminateConstHashJoinCondition)
-                .toRule(RuleType.ELIMINATE_CONST_JOIN_CONDITION);
-    }
-
-    /**
-     * eliminate const hash join condition
-     */
-    public static Plan eliminateConstHashJoinCondition(LogicalJoin<? extends 
Plan, ? extends Plan> join) {
-        List<Expression> newHashConditions = new ArrayList<>();
-        boolean changed = false;
-        for (Expression expr : join.getHashJoinConjuncts()) {
-            boolean eliminate = false;
-            if (expr instanceof EqualTo) {
-                if (((EqualTo) expr).left() instanceof SlotReference
-                        && ((EqualTo) expr).right() instanceof SlotReference) {
-                    EqualTo equal = (EqualTo) 
JoinUtils.swapEqualToForChildrenOrder((EqualTo) expr,
-                            join.left().getOutputSet());
-                    Optional<Expression> leftValue = 
join.left().getLogicalProperties()
-                            .getTrait().getUniformValue((Slot) equal.left());
-
-                    Optional<Expression> rightValue = 
join.right().getLogicalProperties()
-                            .getTrait().getUniformValue((Slot) equal.right());
-                    if (leftValue != null && rightValue != null) {
-                        if (leftValue.isPresent() && rightValue.isPresent()) {
-                            if (leftValue.get().equals(rightValue.get())) {
-                                eliminate = true;
-                                changed = true;
-                            }
-                        }
-                    }
-                }
-                if (!eliminate) {
-                    newHashConditions.add(expr);
-                }
-            }
-        }
-        if (changed) {
-            LOG.info("EliminateConstHashJoinCondition: " + 
join.getHashJoinConjuncts() + " -> " + newHashConditions);
-            return join.withHashJoinConjuncts(newHashConditions);
-        }
-        return join;
-    }
-}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java
index 53f00d89322..fe4f1eed64f 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJoin.java
@@ -434,13 +434,6 @@ public class LogicalJoin<LEFT_CHILD_TYPE extends Plan, 
RIGHT_CHILD_TYPE extends
                 Optional.empty(), Optional.empty(), ImmutableList.of(left, 
right), otherJoinReorderContext);
     }
 
-    public LogicalJoin<Plan, Plan> withHashJoinConjuncts(List<Expression> 
hashJoinConjuncts) {
-        return new LogicalJoin<>(joinType, hashJoinConjuncts, 
otherJoinConjuncts, markJoinConjuncts,
-                hint, markJoinSlotReference, exceptAsteriskOutputs,
-                Optional.empty(), Optional.empty(),
-                ImmutableList.of(left(), right()), joinReorderContext);
-    }
-
     public LogicalJoin<Plan, Plan> withConjunctsChildren(List<Expression> 
hashJoinConjuncts,
             List<Expression> otherJoinConjuncts, Plan left, Plan right, 
JoinReorderContext otherJoinReorderContext) {
         return new LogicalJoin<>(joinType, hashJoinConjuncts, 
otherJoinConjuncts, markJoinConjuncts,
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/PullupExpressionTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/PullupExpressionTest.java
index ff132156441..26daa649ec0 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/PullupExpressionTest.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/PullupExpressionTest.java
@@ -95,7 +95,7 @@ class PullupExpressionTest extends SqlTestBase {
                 .rewrite()
                 .getPlan().child(0);
         CascadesContext c2 = createCascadesContext(
-                "select * from T1 join T2 on T1.id = T2.id where T1.id = 2 and 
T2.id = 1",
+                "select * from T1 join T2 on T1.id = T2.id where T1.id = 1 and 
T2.id = 1",
                 connectContext
         );
         Plan p2 = PlanChecker.from(c2)
@@ -107,12 +107,8 @@ class PullupExpressionTest extends SqlTestBase {
         HyperGraph h2 = HyperGraph.builderForMv(p2).build();
         ComparisonResult res = HyperGraphComparator.isLogicCompatible(h1, h2, 
constructContext(p1, p2, c1));
         Assertions.assertEquals(2, res.getViewExpressions().size());
-        if (res.getViewExpressions().get(0).toSql().equals("(id = 2)")) {
-            Assertions.assertEquals("(id = 1)", 
res.getViewExpressions().get(1).toSql());
-        } else {
-            Assertions.assertEquals("(id = 1)", 
res.getViewExpressions().get(0).toSql());
-            Assertions.assertEquals("(id = 2)", 
res.getViewExpressions().get(1).toSql());
-        }
+        Assertions.assertEquals("(id = 1)", 
res.getViewExpressions().get(0).toSql());
+        Assertions.assertEquals("(id = 1)", 
res.getViewExpressions().get(1).toSql());
     }
 
     @Test
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java
index 3025e7e262f..45e1190412d 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java
@@ -248,7 +248,7 @@ public class MaterializedViewUtilsTest extends 
TestWithFeService {
                 + "\"replication_allocation\" = \"tag.location.default: 1\"\n"
                 + ");\n");
         // Should not make scan to empty relation when the table used by 
materialized view has no data
-        
connectContext.getSessionVariable().setDisableNereidsRules("OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION,ELIMINATE_GROUP_BY_KEY_BY_UNIFORM,ELIMINATE_CONST_JOIN_CONDITION");
+        
connectContext.getSessionVariable().setDisableNereidsRules("OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION,ELIMINATE_GROUP_BY_KEY_BY_UNIFORM");
     }
 
     // Test when join both side are all partition table and partition column 
name is same
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateConstHashJoinConditionTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateConstHashJoinConditionTest.java
deleted file mode 100644
index f9addf9fae3..00000000000
--- 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateConstHashJoinConditionTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-// 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.doris.nereids.rules.rewrite;
-
-import org.apache.doris.nereids.CascadesContext;
-import org.apache.doris.nereids.sqltest.SqlTestBase;
-import org.apache.doris.nereids.util.PlanChecker;
-
-import org.junit.jupiter.api.Test;
-
-public class EliminateConstHashJoinConditionTest extends SqlTestBase {
-
-    @Test
-    void testEliminate() {
-        CascadesContext c1 = createCascadesContext(
-                "select * from T1 join T2 on T1.id = T2.id and T1.id = 1",
-                connectContext
-        );
-        PlanChecker.from(c1)
-                .analyze()
-                .rewrite()
-                .matches(
-                        logicalJoin().when(join ->
-                                join.getHashJoinConjuncts().isEmpty() && 
join.getOtherJoinConjuncts().isEmpty())
-                );
-    }
-
-    @Test
-    void testNotEliminateNonInnerJoin() {
-        CascadesContext c1 = createCascadesContext(
-                "select * from T1 left join T2 on T1.id = T2.id where T1.id = 
1",
-                connectContext
-        );
-        PlanChecker.from(c1)
-                .analyze()
-                .rewrite()
-                .matches(
-                        logicalJoin().when(join ->
-                                !join.getHashJoinConjuncts().isEmpty())
-                );
-    }
-
-}
diff --git a/regression-test/data/nereids_p0/runtime_filter/runtime_filter.out 
b/regression-test/data/nereids_p0/runtime_filter/runtime_filter.out
index f30ad3218cf..6cf5d5a5b4c 100644
Binary files 
a/regression-test/data/nereids_p0/runtime_filter/runtime_filter.out and 
b/regression-test/data/nereids_p0/runtime_filter/runtime_filter.out differ
diff --git 
a/regression-test/data/nereids_rules_p0/infer_predicate/extend_infer_equal_predicate.out
 
b/regression-test/data/nereids_rules_p0/infer_predicate/extend_infer_equal_predicate.out
index 646cd1c6007..980171565da 100644
Binary files 
a/regression-test/data/nereids_rules_p0/infer_predicate/extend_infer_equal_predicate.out
 and 
b/regression-test/data/nereids_rules_p0/infer_predicate/extend_infer_equal_predicate.out
 differ
diff --git 
a/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.out
 
b/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.out
index 0b272a72abe..68562a821b8 100644
Binary files 
a/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.out
 and 
b/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.out
 differ
diff --git 
a/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out
 
b/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out
index 61459617651..bde223ee410 100644
Binary files 
a/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out
 and 
b/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out
 differ
diff --git a/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out 
b/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out
index 5e28fcf402f..be36a1fc265 100644
Binary files a/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out 
and b/regression-test/data/nereids_rules_p0/pkfk/eliminate_inner.out differ
diff --git 
a/regression-test/suites/nereids_p0/cte/test_cte_filter_pushdown.groovy 
b/regression-test/suites/nereids_p0/cte/test_cte_filter_pushdown.groovy
index 21937d3b5c1..528dde0752d 100644
--- a/regression-test/suites/nereids_p0/cte/test_cte_filter_pushdown.groovy
+++ b/regression-test/suites/nereids_p0/cte/test_cte_filter_pushdown.groovy
@@ -20,8 +20,6 @@ suite("test_cte_filter_pushdown") {
     sql "SET enable_fallback_to_original_planner=false"
     sql "set ignore_shape_nodes='PhysicalDistribute, PhysicalProject'"
     sql "set runtime_filter_mode=OFF"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
-
     // CTE filter pushing down with the same filter
     qt_cte_filter_pushdown_1 """
             explain shape plan
diff --git 
a/regression-test/suites/nereids_rules_p0/cte/test_cte_filter_pushdown.groovy 
b/regression-test/suites/nereids_rules_p0/cte/test_cte_filter_pushdown.groovy
index 2ff84fcdb5d..4f28614fdd7 100644
--- 
a/regression-test/suites/nereids_rules_p0/cte/test_cte_filter_pushdown.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/cte/test_cte_filter_pushdown.groovy
@@ -20,7 +20,7 @@ suite("test_cte_filter_pushdown") {
     sql "SET enable_fallback_to_original_planner=false"
     sql "set runtime_filter_type=2;"
     sql "set ignore_shape_nodes='PhysicalDistribute, PhysicalProject'"
-    sql "set 
disable_nereids_rules='PRUNE_EMPTY_PARTITION,ELIMINATE_CONST_JOIN_CONDITION'"
+    sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION"
 
     // CTE filter pushing down with the same filter
     sql """
diff --git 
a/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_through.groovy
 
b/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_through.groovy
index 9607d4dd1a7..e5b6fb32706 100644
--- 
a/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_through.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_through.groovy
@@ -25,7 +25,7 @@ suite("push_filter_through") {
     sql "SET ignore_shape_nodes='PhysicalDistribute, PhysicalProject'"
     sql "set enable_fold_nondeterministic_fn=false"
     sql "set enable_fold_constant_by_be=false"//plan shape will be different
-    sql "set disable_nereids_rules='PRUNE_EMPTY_PARTITION, 
ELIMINATE_CONST_JOIN_CONDITION';"
+    sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION"
 
 
     // push filter through alias
diff --git 
a/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.groovy
 
b/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.groovy
index d436d0a5f2d..bf0d6e74420 100644
--- 
a/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.groovy
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-suite("pull_up_predicate_literal") {
+suite("test_pull_up_predicate_literal") {
     sql """ DROP TABLE IF EXISTS test_pull_up_predicate_literal; """
     sql "set enable_fallback_to_original_planner=false"
     sql """SET ignore_shape_nodes='PhysicalDistribute'"""
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy
index 4f308fe22a8..77f4a5ee45a 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/agg_optimize_when_uniform/agg_optimize_when_uniform.groovy
@@ -18,13 +18,9 @@
 suite("agg_optimize_when_uniform") {
     String db = context.config.getDbNameByFile(context.file)
     sql "use ${db}"
-
-    sql """
-        set enable_agg_state=true;
-        set disable_nereids_rules='ELIMINATE_CONST_JOIN_CONDITION';
-        SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject';
-        set runtime_filter_mode=OFF;
-        """
+    sql "set runtime_filter_mode=OFF";
+    sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'"
+    sql """set enable_agg_state=true"""
 
     sql """
     drop table if exists orders
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy
index 9b094029607..5b8aa6e00d2 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy
@@ -20,7 +20,6 @@ suite("aggregate_with_roll_up") {
     sql "use ${db}"
     sql "set runtime_filter_mode=OFF";
     sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     sql """
     drop table if exists orders
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy
index 6f4e3677b3d..f5545bc41b2 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy
@@ -21,8 +21,7 @@ suite("aggregate_without_roll_up") {
     sql "set runtime_filter_mode=OFF";
     sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'"
     sql "SET enable_agg_state = true"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
-    
+
     sql """
     drop table if exists orders
     """
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy 
b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy
index 195570eb926..59cff69ee89 100644
--- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy
@@ -21,7 +21,6 @@ This suite is a one dimensional test case file.
 suite("partition_mv_rewrite_dimension_1") {
     String db = context.config.getDbNameByFile(context.file)
     sql "use ${db}"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     sql """
     drop table if exists orders_1
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy
index 3810ba3c579..e36a5e277da 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_full_join.groovy
@@ -22,7 +22,6 @@ It mainly tests the full join and filter positions.
 suite("partition_mv_rewrite_dimension_2_full_join") {
     String db = context.config.getDbNameByFile(context.file)
     sql "use ${db}"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     sql """
     drop table if exists orders_2_full_join
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy
index 9cae143e426..44fc259a71a 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_inner_join.groovy
@@ -22,7 +22,6 @@ It mainly tests the inner join and filter positions.
 suite("dimension_2_inner_join") {
     String db = context.config.getDbNameByFile(context.file)
     sql "use ${db}"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     sql """
     drop table if exists orders_2_2
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy
index 7dd3149bf75..61670741295 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_left_join.groovy
@@ -22,7 +22,6 @@ It mainly tests the left join and filter positions.
 suite("partition_mv_rewrite_dimension_2_1") {
     String db = context.config.getDbNameByFile(context.file)
     sql "use ${db}"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     sql """
     drop table if exists orders_2_1
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy
index fdb1ff3c2c6..850de42fc69 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_2_right_join.groovy
@@ -22,7 +22,6 @@ It mainly tests the right join and filter positions.
 suite("partition_mv_rewrite_dimension_2_right_join") {
     String db = context.config.getDbNameByFile(context.file)
     sql "use ${db}"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     sql """
     drop table if exists orders_2_right_join
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy
index 93e438d3f24..8c0674faeb8 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_inner/inner_join_dphyp.groovy
@@ -20,7 +20,6 @@ suite("inner_join_dphyp") {
     sql "use ${db}"
     sql "set runtime_filter_mode=OFF"
     sql "SET enable_dphyp_optimizer = true"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     sql """
     drop table if exists orders
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy
index 8cbddc34c74..86c668a9c0c 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/join/dphyp_outer/outer_join_dphyp.groovy
@@ -21,8 +21,6 @@ suite("outer_join_dphyp") {
     sql "set runtime_filter_mode=OFF";
     sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'"
     sql "SET enable_dphyp_optimizer = true"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
-
     sql """
     drop table if exists orders
     """
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy 
b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy
index 1244b730877..3aebf92b7ce 100644
--- a/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy
@@ -19,7 +19,6 @@ suite("inner_join") {
     String db = context.config.getDbNameByFile(context.file)
     sql "use ${db}"
     sql "set runtime_filter_mode=OFF"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     sql """
     drop table if exists orders
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy 
b/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy
index c89d466fcf1..12c03922c8e 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy
@@ -20,7 +20,6 @@ suite("outer_join") {
     sql "use ${db}"
     sql "set runtime_filter_mode=OFF";
     sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     sql """
     drop table if exists orders
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy
index 15709ed1f6f..cce090b2a94 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/join_infer_derive/inner_join_infer_and_derive.groovy
@@ -26,7 +26,6 @@ suite("inner_join_infer_and_derive") {
     sql "SET enable_fallback_to_original_planner=false"
     sql "SET enable_materialized_view_rewrite=true"
     sql "SET enable_nereids_timeout = false"
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     sql """
     drop table if exists orders_inner
diff --git 
a/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy
 
b/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy
index f26e19e721a..692ea03fc61 100644
--- 
a/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/mv/nested/nested_materialized_view.groovy
@@ -17,7 +17,6 @@
 
 suite("nested_materialized_view") {
 
-    sql "set disable_nereids_rules=ELIMINATE_CONST_JOIN_CONDITION"
 
     def compare_res = { def stmt ->
         sql "SET enable_materialized_view_rewrite=false"
diff --git 
a/regression-test/suites/nereids_rules_p0/predicate_infer/infer_predicate.groovy
 
b/regression-test/suites/nereids_rules_p0/predicate_infer/infer_predicate.groovy
index 5e4207fa2e0..6237d5d66e8 100644
--- 
a/regression-test/suites/nereids_rules_p0/predicate_infer/infer_predicate.groovy
+++ 
b/regression-test/suites/nereids_rules_p0/predicate_infer/infer_predicate.groovy
@@ -21,7 +21,7 @@ suite("infer_predicate") {
     sql "SET enable_fallback_to_original_planner=false"
     sql "SET disable_join_reorder=true"
     sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'"
-    sql "set 
disable_nereids_rules='PRUNE_EMPTY_PARTITION,ELIMINATE_CONST_JOIN_CONDITION'"
+    sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION"
 
 
     sql """


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org
For additional commands, e-mail: commits-h...@doris.apache.org


Reply via email to