You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2024/04/10 07:00:15 UTC

(doris) branch 2.1-tmp updated (6c5dd820c0a -> 9bc7902e5a0)

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

yiguolei pushed a change to branch 2.1-tmp
in repository https://gitbox.apache.org/repos/asf/doris.git


    from 6c5dd820c0a [improvement](spill) improve spill timers (#33156)
     new ff990eb8693 [enhancement](Nereids) refactor expression rewriter to pattern match (#32617)
     new 38d580dfb79 [fix](Nereids) fix link children failed (#33134)
     new cc363f26c26 [fix](Nereids) fix group concat (#33091)
     new 9bc7902e5a0 [fix](Nereids) fix bind group by int literal (#33117)

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/pipeline/pipeline_fragment_context.cpp      |   9 +-
 .../pipeline_x/pipeline_x_fragment_context.cpp     |  11 +-
 be/src/runtime/descriptors.h                       |   5 +
 be/src/vec/exec/vaggregation_node.h                |   1 +
 fe/fe-core/pom.xml                                 |   2 +-
 .../org/apache/doris/analysis/DateLiteral.java     |   9 +-
 .../java/org/apache/doris/catalog/OlapTable.java   |   5 +-
 .../org/apache/doris/mtmv/MTMVRelationManager.java |   4 +-
 .../org/apache/doris/mysql/privilege/Role.java     |   4 +-
 .../org/apache/doris/nereids/CascadesContext.java  |  49 +++-
 .../org/apache/doris/nereids/NereidsPlanner.java   |   2 +-
 .../org/apache/doris/nereids/StatementContext.java |  15 ++
 .../org/apache/doris/nereids/analyzer/Scope.java   |  19 +-
 .../java/org/apache/doris/nereids/jobs/Job.java    |  11 +-
 .../doris/nereids/jobs/executor/Analyzer.java      |   2 -
 .../doris/nereids/jobs/executor/Rewriter.java      |   7 +-
 .../jobs/joinorder/hypergraph/HyperGraph.java      |  11 +-
 .../nereids/jobs/rewrite/CustomRewriteJob.java     |   6 +-
 .../jobs/rewrite/PlanTreeRewriteBottomUpJob.java   | 113 ++++++----
 .../nereids/jobs/rewrite/PlanTreeRewriteJob.java   |  65 ++++--
 .../jobs/rewrite/PlanTreeRewriteTopDownJob.java    |  41 +++-
 .../nereids/jobs/rewrite/RewriteJobContext.java    |  10 +-
 .../jobs/rewrite/RootPlanTreeRewriteJob.java       |  16 +-
 .../java/org/apache/doris/nereids/memo/Group.java  |  22 +-
 .../apache/doris/nereids/memo/GroupExpression.java |   5 +
 .../nereids/pattern/ExpressionPatternRules.java    | 112 +++++++++
 .../ExpressionPatternTraverseListeners.java        | 112 +++++++++
 .../doris/nereids/pattern/ParentTypeIdMapping.java |  59 +++++
 .../org/apache/doris/nereids/pattern/Pattern.java  |   4 +
 .../apache/doris/nereids/pattern/TypeMappings.java | 133 +++++++++++
 .../generator/ExpressionTypeMappingGenerator.java  | 159 +++++++++++++
 ...GeneratorAnalyzer.java => JavaAstAnalyzer.java} |  93 ++++----
 .../generator/LogicalBinaryPatternGenerator.java   |   4 +-
 .../generator/LogicalLeafPatternGenerator.java     |   4 +-
 .../generator/LogicalUnaryPatternGenerator.java    |   4 +-
 .../generator/PatternDescribableProcessor.java     |  34 ++-
 .../generator/PhysicalBinaryPatternGenerator.java  |   4 +-
 .../generator/PhysicalLeafPatternGenerator.java    |   4 +-
 .../generator/PhysicalUnaryPatternGenerator.java   |   4 +-
 ...ernGenerator.java => PlanPatternGenerator.java} |  18 +-
 .../generator/PlanPatternGeneratorAnalyzer.java    |  73 ++++++
 .../generator/PlanTypeMappingGenerator.java        | 159 +++++++++++++
 .../processor/post/RuntimeFilterPruner.java        |  17 +-
 .../doris/nereids/processor/post/Validator.java    |  10 +-
 .../properties/ChildrenPropertiesRegulator.java    |   4 +-
 .../nereids/properties/FunctionalDependencies.java |  24 +-
 .../nereids/properties/LogicalProperties.java      |  50 +++--
 .../nereids/properties/PhysicalProperties.java     |   6 +-
 .../java/org/apache/doris/nereids/rules/Rule.java  |   6 +-
 .../org/apache/doris/nereids/rules/RuleSet.java    |   4 +-
 .../AdjustAggregateNullableForEmptySet.java        |  29 ++-
 .../nereids/rules/analysis/BindExpression.java     |  61 +++--
 .../nereids/rules/analysis/BindSlotWithPaths.java  |  29 +--
 .../nereids/rules/analysis/CheckAfterRewrite.java  |  85 +++----
 .../nereids/rules/analysis/CheckAnalysis.java      |  36 +--
 .../rules/analysis/EliminateGroupByConstant.java   |   2 +-
 .../nereids/rules/analysis/ExpressionAnalyzer.java |   2 +-
 .../nereids/rules/analysis/FillUpMissingSlots.java |  21 +-
 .../nereids/rules/analysis/NormalizeAggregate.java |  48 ++--
 .../analysis/ReplaceExpressionByChildOutput.java   |  48 ++--
 .../ResolveOrdinalInOrderByAndGroupBy.java         | 102 ---------
 .../nereids/rules/analysis/SubqueryToApply.java    |  77 +++++--
 .../mv/AbstractMaterializedViewRule.java           |  15 +-
 .../mv/InitMaterializationContextHook.java         |   4 +-
 .../exploration/mv/MaterializationContext.java     |   8 +-
 .../nereids/rules/exploration/mv/StructInfo.java   |  20 +-
 .../expression/ExpressionBottomUpRewriter.java     | 124 ++++++++++
 .../expression/ExpressionListenerMatcher.java      |  41 ++++
 ...eContext.java => ExpressionMatchingAction.java} |  16 +-
 ...Context.java => ExpressionMatchingContext.java} |  25 ++-
 .../rules/expression/ExpressionNormalization.java  |  29 +--
 ...=> ExpressionNormalizationAndOptimization.java} |  22 +-
 .../rules/expression/ExpressionOptimization.java   |  26 ++-
 .../expression/ExpressionPatternMatchRule.java     |  64 ++++++
 .../rules/expression/ExpressionPatternMatcher.java |  41 ++++
 .../expression/ExpressionPatternRuleFactory.java   |  84 +++++++
 .../rules/expression/ExpressionRewrite.java        |  51 ++++-
 .../rules/expression/ExpressionRewriteContext.java |   4 +-
 .../rules/expression/ExpressionRuleExecutor.java   |  16 +-
 ...ontext.java => ExpressionTraverseListener.java} |  16 +-
 .../ExpressionTraverseListenerFactory.java         |  79 +++++++
 .../ExpressionTraverseListenerMapping.java         |  59 +++++
 .../nereids/rules/expression/check/CheckCast.java  |  24 +-
 .../rules/ArrayContainToArrayOverlap.java          |  94 +++++---
 .../rules/expression/rules/CaseWhenToIf.java       |  18 +-
 .../expression/rules/ConvertAggStateCast.java      |  33 ++-
 .../expression/rules/DateFunctionRewrite.java      |  34 ++-
 .../expression/rules/DigitalMaskingConvert.java    |  23 +-
 .../expression/rules/DistinctPredicatesRule.java   |  18 +-
 .../expression/rules/ExtractCommonFactorRule.java  | 222 +++++++++++++++---
 .../rules/expression/rules/FoldConstantRule.java   |  32 ++-
 .../expression/rules/FoldConstantRuleOnBE.java     |  46 ++--
 .../expression/rules/FoldConstantRuleOnFE.java     | 170 ++++++++++++--
 .../rules/expression/rules/InPredicateDedup.java   |  40 ++--
 .../expression/rules/InPredicateToEqualToRule.java |  25 ++-
 .../rules/NormalizeBinaryPredicatesRule.java       |  21 +-
 .../expression/rules/NullSafeEqualToEqual.java     |  21 +-
 .../rules/OneListPartitionEvaluator.java           |   2 +-
 .../rules/OneRangePartitionEvaluator.java          | 120 ++++++----
 .../nereids/rules/expression/rules/OrToIn.java     |  36 +--
 .../rules/expression/rules/PartitionPruner.java    |  23 +-
 .../expression/rules/PartitionRangeExpander.java   | 115 ++++++----
 .../rules/PredicateRewriteForPartitionPrune.java   |   4 +-
 .../rules/RangePartitionValueIterator.java         |  64 ++++++
 .../expression/rules/ReplaceVariableByLiteral.java |  17 +-
 .../rules/SimplifyArithmeticComparisonRule.java    | 105 +++++----
 .../expression/rules/SimplifyArithmeticRule.java   |  70 +++---
 .../rules/expression/rules/SimplifyCastRule.java   |  21 +-
 .../rules/SimplifyComparisonPredicate.java         |  37 ++-
 .../rules/SimplifyDecimalV3Comparison.java         |  24 +-
 .../expression/rules/SimplifyInPredicate.java      |  20 +-
 .../expression/rules/SimplifyNotExprRule.java      |  34 ++-
 .../rules/expression/rules/SimplifyRange.java      |  73 +++---
 .../expression/rules/SupportJavaDateFormatter.java |  44 ++--
 .../nereids/rules/expression/rules/TopnToMax.java  |  29 ++-
 .../rules/TryEliminateUninterestedPredicates.java  |  14 +-
 .../rules/implementation/AggregateStrategies.java  |  34 ++-
 .../rules/rewrite/AdjustConjunctsReturnType.java   |   4 +-
 .../nereids/rules/rewrite/AdjustNullable.java      |  12 +-
 .../rules/rewrite/CheckMatchExpression.java        |   7 +-
 .../nereids/rules/rewrite/CheckPrivileges.java     |  29 ++-
 .../doris/nereids/rules/rewrite/ColumnPruning.java |  98 ++++----
 .../rules/rewrite/CountDistinctRewrite.java        |  60 ++---
 .../nereids/rules/rewrite/CountLiteralRewrite.java |  37 ++-
 .../nereids/rules/rewrite/EliminateFilter.java     |   7 +-
 .../nereids/rules/rewrite/EliminateGroupBy.java    |  56 +++--
 .../nereids/rules/rewrite/EliminateMarkJoin.java   |  17 +-
 .../nereids/rules/rewrite/EliminateNotNull.java    |  39 ++--
 .../rules/rewrite/EliminateOrderByConstant.java    |  16 +-
 .../ExtractAndNormalizeWindowExpression.java       | 161 +++++++------
 ...xtractSingleTableExpressionFromDisjunction.java |   9 +-
 .../nereids/rules/rewrite/InferJoinNotNull.java    |   4 +-
 .../nereids/rules/rewrite/MergeAggregate.java      |   2 +-
 .../doris/nereids/rules/rewrite/MergeProjects.java |  10 +-
 .../doris/nereids/rules/rewrite/NormalizeSort.java |  59 +++--
 .../nereids/rules/rewrite/NormalizeToSlot.java     |  43 ++--
 .../rules/rewrite/PruneOlapScanPartition.java      |  51 +++--
 .../nereids/rules/rewrite/PullUpPredicates.java    |  75 ++++---
 .../rewrite/PushDownFilterThroughAggregation.java  |  12 +-
 .../rewrite/PushDownFilterThroughProject.java      |  13 +-
 .../doris/nereids/rules/rewrite/ReorderJoin.java   |   6 +-
 .../nereids/rules/rewrite/SimplifyAggGroupBy.java  |  23 +-
 .../mv/AbstractSelectMaterializedIndexRule.java    |  15 +-
 .../mv/SelectMaterializedIndexWithAggregate.java   |   9 +-
 .../SelectMaterializedIndexWithoutAggregate.java   |  45 ++--
 .../doris/nereids/stats/StatsCalculator.java       |  11 +-
 .../doris/nereids/trees/AbstractTreeNode.java      |  22 +-
 .../org/apache/doris/nereids/trees/TreeNode.java   |  17 ++
 .../nereids/trees/expressions/BinaryOperator.java  |   6 -
 .../trees/expressions/ComparisonPredicate.java     |   4 +-
 .../nereids/trees/expressions/Expression.java      |  99 +++++---
 .../nereids/trees/expressions/InPredicate.java     |   5 +-
 .../nereids/trees/expressions/SlotReference.java   |   7 +-
 .../functions/ComputeSignatureHelper.java          |  11 +-
 .../functions/agg/AggregateFunction.java           |  17 +-
 .../scalar/PushDownToProjectionFunction.java       |   7 +-
 .../trees/expressions/literal/DateLiteral.java     |  39 +++-
 .../visitor/DefaultExpressionRewriter.java         |  10 +-
 .../doris/nereids/trees/plans/AbstractPlan.java    |  28 +--
 .../org/apache/doris/nereids/trees/plans/Plan.java |  61 +++--
 .../nereids/trees/plans/algebra/Aggregate.java     |  17 +-
 .../doris/nereids/trees/plans/algebra/Project.java |  27 ++-
 .../trees/plans/logical/LogicalAggregate.java      |   8 +-
 .../plans/logical/LogicalCatalogRelation.java      | 132 ++++++-----
 .../trees/plans/logical/LogicalOlapScan.java       |  65 ++++--
 .../trees/plans/logical/LogicalProject.java        |   8 +-
 .../nereids/trees/plans/logical/LogicalSort.java   |  19 +-
 .../nereids/trees/plans/logical/LogicalTopN.java   |  13 +-
 .../trees/plans/physical/PhysicalHashJoin.java     |   3 +-
 .../apache/doris/nereids/util/ExpressionUtils.java | 250 ++++++++++++++++-----
 .../doris/nereids/util/ImmutableEqualSet.java      |   6 +-
 .../org/apache/doris/nereids/util/JoinUtils.java   |   5 +-
 .../org/apache/doris/nereids/util/PlanUtils.java   |  24 ++
 .../doris/nereids/util/TypeCoercionUtils.java      |  19 +-
 .../java/org/apache/doris/nereids/util/Utils.java  |  65 +++++-
 .../java/org/apache/doris/qe/SessionVariable.java  |  38 +++-
 .../jobs/joinorder/hypergraph/HyperGraphTest.java  |  12 +-
 .../rules/expression/ExpressionRewriteTest.java    |  80 +++++--
 .../expression/ExpressionRewriteTestHelper.java    |   2 +-
 .../nereids/rules/expression/FoldConstantTest.java |  36 ++-
 .../rules/expression/PredicatesSplitterTest.java   |   2 +-
 .../expression/SimplifyArithmeticRuleTest.java     |  56 +++--
 .../rules/expression/SimplifyInPredicateTest.java  |   8 +-
 .../rules/expression/SimplifyRangeTest.java        |  26 ++-
 .../expression/rules/NullSafeEqualToEqualTest.java |  20 +-
 .../SimplifyArithmeticComparisonRuleTest.java      |   7 +-
 .../expression/rules/SimplifyCastRuleTest.java     |   7 +-
 .../rules/SimplifyComparisonPredicateTest.java     |  35 ++-
 .../rules/SimplifyDecimalV3ComparisonTest.java     |   6 +-
 .../rules/expression/rules/TopnToMaxTest.java      |   4 +-
 .../rules/rewrite/EliminateJoinByFkTest.java       |   1 +
 .../doris/nereids/rules/rewrite/OrToInTest.java    |  19 +-
 .../PushDownFilterThroughAggregationTest.java      |   4 +-
 .../functions/ComputeSignatureHelperTest.java      |  11 +
 .../nereids/trees/plans/SetOperationTest.java      |  38 ++++
 .../org/apache/doris/nereids/util/PlanChecker.java |  20 ++
 .../data/nereids_hint_tpcds_p0/shape/query24.out   |  10 +-
 .../data/nereids_hint_tpcds_p0/shape/query64.out   |   2 +-
 .../filter_push_down/push_filter_through.out       |  28 +--
 .../data/nereids_syntax_p0/bind_priority.out       |   6 +
 .../shape/query13.out                              |   2 +-
 .../shape/query14.out                              |   2 +-
 .../shape/query24.out                              |   4 +-
 .../shape/query41.out                              |   2 +-
 .../shape/query50.out                              |   2 +-
 .../shape/query64.out                              |   2 +-
 .../shape/query85.out                              |   2 +-
 .../shape/query95.out                              |   2 +-
 .../noStatsRfPrune/query13.out                     |   2 +-
 .../noStatsRfPrune/query17.out                     |   2 +-
 .../noStatsRfPrune/query41.out                     |   2 +-
 .../noStatsRfPrune/query47.out                     |   7 +-
 .../noStatsRfPrune/query50.out                     |   2 +-
 .../noStatsRfPrune/query57.out                     |   7 +-
 .../noStatsRfPrune/query6.out                      |  57 +++--
 .../noStatsRfPrune/query65.out                     |   2 +-
 .../no_stats_shape/query13.out                     |   2 +-
 .../no_stats_shape/query14.out                     |   2 +-
 .../no_stats_shape/query17.out                     |   2 +-
 .../no_stats_shape/query24.out                     |   2 +-
 .../no_stats_shape/query41.out                     |   2 +-
 .../no_stats_shape/query47.out                     |   7 +-
 .../no_stats_shape/query50.out                     |   2 +-
 .../no_stats_shape/query57.out                     |   7 +-
 .../no_stats_shape/query6.out                      |  57 +++--
 .../no_stats_shape/query64.out                     |   2 +-
 .../no_stats_shape/query65.out                     |   2 +-
 .../no_stats_shape/query85.out                     |   6 +-
 .../rf_prune/query13.out                           |   2 +-
 .../rf_prune/query14.out                           |   2 +-
 .../rf_prune/query41.out                           |   2 +-
 .../rf_prune/query50.out                           |   2 +-
 .../rf_prune/query85.out                           |   2 +-
 .../rf_prune/query95.out                           |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query13.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query14.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query24.out |   4 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query41.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query50.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query64.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query85.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query95.out |   2 +-
 .../data/nereids_tpch_shape_sf1000_p0/shape/q9.out |   2 +-
 .../shape_no_stats/q9.out                          |   2 +-
 .../org/apache/doris/regression/suite/Suite.groovy |   7 +-
 .../doris/regression/util/OutputUtils.groovy       |  28 ++-
 .../doris/regression/util/ReusableIterator.groovy  |   7 +
 .../suites/nereids_syntax_p0/bind_priority.groovy  |  28 +++
 248 files changed, 5153 insertions(+), 2081 deletions(-)
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ExpressionPatternRules.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ExpressionPatternTraverseListeners.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ParentTypeIdMapping.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/TypeMappings.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/ExpressionTypeMappingGenerator.java
 rename fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/{PatternGeneratorAnalyzer.java => JavaAstAnalyzer.java} (75%)
 rename fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/{PatternGenerator.java => PlanPatternGenerator.java} (96%)
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGeneratorAnalyzer.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanTypeMappingGenerator.java
 delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionBottomUpRewriter.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionListenerMatcher.java
 copy fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/{ExpressionRewriteContext.java => ExpressionMatchingAction.java} (73%)
 copy fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/{ExpressionRewriteContext.java => ExpressionMatchingContext.java} (55%)
 copy fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/{ExpressionRewriteContext.java => ExpressionNormalizationAndOptimization.java} (59%)
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternMatchRule.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternMatcher.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternRuleFactory.java
 copy fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/{ExpressionRewriteContext.java => ExpressionTraverseListener.java} (66%)
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListenerFactory.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListenerMapping.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/RangePartitionValueIterator.java


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


(doris) 03/04: [fix](Nereids) fix group concat (#33091)

Posted by yi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yiguolei pushed a commit to branch 2.1-tmp
in repository https://gitbox.apache.org/repos/asf/doris.git

commit cc363f26c26515fe4c06cac0f9fca9b0500cc045
Author: 924060929 <92...@qq.com>
AuthorDate: Mon Apr 1 21:28:39 2024 +0800

    [fix](Nereids) fix group concat (#33091)
    
    Fix failed in regression_test/suites/query_p0/group_concat/test_group_concat.groovy
    
    select
    group_concat( distinct b1, '?'), group_concat( distinct b3, '?')
    from
    table_group_concat
    group by
    b2
    
    exception:
    
    lowestCostPlans with physicalProperties(GATHER) doesn't exist in root group
    
    The root cause is '?' is push down to slot by NormalizeAggregate, AggregateStrategies treat the slot as a distinct parameter and generate a invalid PhysicalHashAggregate, and then reject by ChildOutputPropertyDeriver.
    
    I fix this bug by avoid push down literal to slot in NormalizeAggregate, and forbidden generate stream aggregate node when group by slots is empty
---
 be/src/pipeline/pipeline_fragment_context.cpp      |  9 ++++--
 .../pipeline_x/pipeline_x_fragment_context.cpp     | 11 ++++++--
 be/src/runtime/descriptors.h                       |  5 ++++
 be/src/vec/exec/vaggregation_node.h                |  1 +
 .../java/org/apache/doris/nereids/memo/Group.java  | 22 +++++++++++++--
 .../apache/doris/nereids/memo/GroupExpression.java |  5 ++++
 .../properties/ChildrenPropertiesRegulator.java    |  4 ++-
 .../nereids/properties/PhysicalProperties.java     |  6 ++--
 .../nereids/rules/analysis/NormalizeAggregate.java |  9 +++++-
 .../rules/implementation/AggregateStrategies.java  | 32 ++++++++++++++++++++++
 10 files changed, 93 insertions(+), 11 deletions(-)

diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp
index c273a0c3807..a32d777788d 100644
--- a/be/src/pipeline/pipeline_fragment_context.cpp
+++ b/be/src/pipeline/pipeline_fragment_context.cpp
@@ -559,7 +559,12 @@ Status PipelineFragmentContext::_build_pipelines(ExecNode* node, PipelinePtr cur
         auto* agg_node = dynamic_cast<vectorized::AggregationNode*>(node);
         auto new_pipe = add_pipeline();
         RETURN_IF_ERROR(_build_pipelines(node->child(0), new_pipe));
-        if (agg_node->is_aggregate_evaluators_empty()) {
+        if (agg_node->is_probe_expr_ctxs_empty() && node->row_desc().num_slots() == 0) {
+            return Status::InternalError("Illegal aggregate node " +
+                                         std::to_string(agg_node->id()) +
+                                         ": group by and output is empty");
+        }
+        if (agg_node->is_aggregate_evaluators_empty() && !agg_node->is_probe_expr_ctxs_empty()) {
             auto data_queue = std::make_shared<DataQueue>(1);
             OperatorBuilderPtr pre_agg_sink =
                     std::make_shared<DistinctStreamingAggSinkOperatorBuilder>(node->id(), agg_node,
@@ -570,7 +575,7 @@ Status PipelineFragmentContext::_build_pipelines(ExecNode* node, PipelinePtr cur
                     std::make_shared<DistinctStreamingAggSourceOperatorBuilder>(
                             node->id(), agg_node, data_queue);
             RETURN_IF_ERROR(cur_pipe->add_operator(pre_agg_source));
-        } else if (agg_node->is_streaming_preagg()) {
+        } else if (agg_node->is_streaming_preagg() && !agg_node->is_probe_expr_ctxs_empty()) {
             auto data_queue = std::make_shared<DataQueue>(1);
             OperatorBuilderPtr pre_agg_sink = std::make_shared<StreamingAggSinkOperatorBuilder>(
                     node->id(), agg_node, data_queue);
diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp
index 744ce754a59..5dac71e8420 100644
--- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp
+++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp
@@ -989,13 +989,20 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN
         break;
     }
     case TPlanNodeType::AGGREGATION_NODE: {
+        if (tnode.agg_node.grouping_exprs.empty() &&
+            descs.get_tuple_descriptor(tnode.agg_node.output_tuple_id)->slots().empty()) {
+            return Status::InternalError("Illegal aggregate node " + std::to_string(tnode.node_id) +
+                                         ": group by and output is empty");
+        }
         if (tnode.agg_node.aggregate_functions.empty() && !_runtime_state->enable_agg_spill() &&
             request.query_options.__isset.enable_distinct_streaming_aggregation &&
-            request.query_options.enable_distinct_streaming_aggregation) {
+            request.query_options.enable_distinct_streaming_aggregation &&
+            !tnode.agg_node.grouping_exprs.empty()) {
             op.reset(new DistinctStreamingAggOperatorX(pool, next_operator_id(), tnode, descs));
             RETURN_IF_ERROR(cur_pipe->add_operator(op));
         } else if (tnode.agg_node.__isset.use_streaming_preaggregation &&
-                   tnode.agg_node.use_streaming_preaggregation) {
+                   tnode.agg_node.use_streaming_preaggregation &&
+                   !tnode.agg_node.grouping_exprs.empty()) {
             op.reset(new StreamingAggOperatorX(pool, next_operator_id(), tnode, descs));
             RETURN_IF_ERROR(cur_pipe->add_operator(op));
         } else {
diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h
index fff1ed339d5..7cb7e9fe015 100644
--- a/be/src/runtime/descriptors.h
+++ b/be/src/runtime/descriptors.h
@@ -505,10 +505,12 @@ public:
               _has_varlen_slots(desc._has_varlen_slots) {
         _num_materialized_slots = 0;
         _num_null_slots = 0;
+        _num_slots = 0;
         std::vector<TupleDescriptor*>::const_iterator it = desc._tuple_desc_map.begin();
         for (; it != desc._tuple_desc_map.end(); ++it) {
             _num_materialized_slots += (*it)->num_materialized_slots();
             _num_null_slots += (*it)->num_null_slots();
+            _num_slots += (*it)->slots().size();
         }
         _num_null_bytes = (_num_null_slots + 7) / 8;
     }
@@ -531,6 +533,8 @@ public:
 
     int num_null_bytes() const { return _num_null_bytes; }
 
+    int num_slots() const { return _num_slots; }
+
     static const int INVALID_IDX;
 
     // Returns INVALID_IDX if id not part of this row.
@@ -585,6 +589,7 @@ private:
     int _num_materialized_slots;
     int _num_null_slots;
     int _num_null_bytes;
+    int _num_slots;
 };
 
 } // namespace doris
diff --git a/be/src/vec/exec/vaggregation_node.h b/be/src/vec/exec/vaggregation_node.h
index f09ebcbba83..f89bbb9d780 100644
--- a/be/src/vec/exec/vaggregation_node.h
+++ b/be/src/vec/exec/vaggregation_node.h
@@ -416,6 +416,7 @@ public:
     Status pull(doris::RuntimeState* state, vectorized::Block* output_block, bool* eos) override;
     Status sink(doris::RuntimeState* state, vectorized::Block* input_block, bool eos) override;
     Status do_pre_agg(vectorized::Block* input_block, vectorized::Block* output_block);
+    bool is_probe_expr_ctxs_empty() const { return _probe_expr_ctxs.empty(); }
     bool is_streaming_preagg() const { return _is_streaming_preagg; }
     bool is_aggregate_evaluators_empty() const { return _aggregate_evaluators.empty(); }
     void _make_nullable_output_key(Block* block);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java
index 5a5abd56f95..01968a03bef 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java
@@ -35,6 +35,7 @@ import org.apache.doris.statistics.Statistics;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -65,7 +66,7 @@ public class Group {
 
     // Map of cost lower bounds
     // Map required plan props to cost lower bound of corresponding plan
-    private final Map<PhysicalProperties, Pair<Cost, GroupExpression>> lowestCostPlans = Maps.newHashMap();
+    private final Map<PhysicalProperties, Pair<Cost, GroupExpression>> lowestCostPlans = Maps.newLinkedHashMap();
 
     private boolean isExplored = false;
 
@@ -228,6 +229,12 @@ public class Group {
         return costAndGroupExpression;
     }
 
+    public Map<PhysicalProperties, Cost> getLowestCosts() {
+        return lowestCostPlans.entrySet()
+                .stream()
+                .collect(ImmutableMap.toImmutableMap(Entry::getKey, kv -> kv.getValue().first));
+    }
+
     public GroupExpression getBestPlan(PhysicalProperties properties) {
         if (lowestCostPlans.containsKey(properties)) {
             return lowestCostPlans.get(properties).second;
@@ -489,9 +496,18 @@ public class Group {
     public String treeString() {
         Function<Object, String> toString = obj -> {
             if (obj instanceof Group) {
-                return "Group[" + ((Group) obj).groupId + "]";
+                Group group = (Group) obj;
+                Map<PhysicalProperties, Cost> lowestCosts = group.getLowestCosts();
+                return "Group[" + group.groupId + ", lowestCosts: " + lowestCosts + "]";
             } else if (obj instanceof GroupExpression) {
-                return ((GroupExpression) obj).getPlan().toString();
+                GroupExpression groupExpression = (GroupExpression) obj;
+                Map<PhysicalProperties, Pair<Cost, List<PhysicalProperties>>> lowestCostTable
+                        = groupExpression.getLowestCostTable();
+                Map<PhysicalProperties, PhysicalProperties> requestPropertiesMap
+                        = groupExpression.getRequestPropertiesMap();
+                Cost cost = groupExpression.getCost();
+                return groupExpression.getPlan().toString() + " [cost: " + cost + ", lowestCostTable: "
+                        + lowestCostTable + ", requestPropertiesMap: " + requestPropertiesMap + "]";
             } else if (obj instanceof Pair) {
                 // print logicalExpressions or physicalExpressions
                 // first is name, second is group expressions
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java
index eda7f5c9c35..24bc9383b52 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java
@@ -35,6 +35,7 @@ import org.apache.doris.statistics.Statistics;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -318,6 +319,10 @@ public class GroupExpression {
         this.estOutputRowCount = estOutputRowCount;
     }
 
+    public Map<PhysicalProperties, PhysicalProperties> getRequestPropertiesMap() {
+        return ImmutableMap.copyOf(requestPropertiesMap);
+    }
+
     @Override
     public String toString() {
         DecimalFormat format = new DecimalFormat("#,###.##");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java
index 7c5374ebd21..366730f7dc5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java
@@ -104,6 +104,9 @@ public class ChildrenPropertiesRegulator extends PlanVisitor<Boolean, Void> {
 
     @Override
     public Boolean visitPhysicalHashAggregate(PhysicalHashAggregate<? extends Plan> agg, Void context) {
+        if (agg.getGroupByExpressions().isEmpty() && agg.getOutputExpressions().isEmpty()) {
+            return false;
+        }
         if (!agg.getAggregateParam().canBeBanned) {
             return true;
         }
@@ -121,7 +124,6 @@ public class ChildrenPropertiesRegulator extends PlanVisitor<Boolean, Void> {
                 return true;
             }
             return false;
-
         }
 
         // forbid TWO_PHASE_AGGREGATE_WITH_DISTINCT after shuffle
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java
index 81e7190e163..031f18ab918 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java
@@ -88,11 +88,13 @@ public class PhysicalProperties {
                 .map(SlotReference.class::cast)
                 .map(SlotReference::getExprId)
                 .collect(Collectors.toList());
-        return createHash(partitionedSlots, shuffleType);
+        return partitionedSlots.isEmpty() ? PhysicalProperties.GATHER : createHash(partitionedSlots, shuffleType);
     }
 
     public static PhysicalProperties createHash(List<ExprId> orderedShuffledColumns, ShuffleType shuffleType) {
-        return new PhysicalProperties(new DistributionSpecHash(orderedShuffledColumns, shuffleType));
+        return orderedShuffledColumns.isEmpty()
+                ? PhysicalProperties.GATHER
+                : new PhysicalProperties(new DistributionSpecHash(orderedShuffledColumns, shuffleType));
     }
 
     public static PhysicalProperties createHash(DistributionSpecHash distributionSpecHash) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java
index 7f6df51248e..e9b3d32da6e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java
@@ -31,6 +31,7 @@ import org.apache.doris.nereids.trees.expressions.SlotReference;
 import org.apache.doris.nereids.trees.expressions.SubqueryExpr;
 import org.apache.doris.nereids.trees.expressions.WindowExpression;
 import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
+import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
 import org.apache.doris.nereids.trees.plans.logical.LogicalHaving;
@@ -152,6 +153,9 @@ public class NormalizeAggregate implements RewriteRuleFactory, NormalizeToSlot {
         Map<Boolean, ImmutableSet<Expression>> categorizedNoDistinctAggsChildren = aggFuncs.stream()
                 .filter(aggFunc -> !aggFunc.isDistinct())
                 .flatMap(agg -> agg.children().stream())
+                // should not push down literal under aggregate
+                // e.g. group_concat(distinct xxx, ','), the ',' literal show stay in aggregate
+                .filter(arg -> !(arg instanceof Literal))
                 .collect(Collectors.groupingBy(
                         child -> child.containsType(SubqueryExpr.class, WindowExpression.class),
                         ImmutableSet.toImmutableSet()));
@@ -159,9 +163,12 @@ public class NormalizeAggregate implements RewriteRuleFactory, NormalizeToSlot {
         // split distinct agg child as two parts
         // TRUE part 1: need push down itself, if it is NOT SlotReference or Literal
         // FALSE part 2: need push down its input slots, if it is SlotReference or Literal
-        Map<Object, ImmutableSet<Expression>> categorizedDistinctAggsChildren = aggFuncs.stream()
+        Map<Boolean, ImmutableSet<Expression>> categorizedDistinctAggsChildren = aggFuncs.stream()
                 .filter(AggregateFunction::isDistinct)
                 .flatMap(agg -> agg.children().stream())
+                // should not push down literal under aggregate
+                // e.g. group_concat(distinct xxx, ','), the ',' literal show stay in aggregate
+                .filter(arg -> !(arg instanceof Literal))
                 .collect(
                         Collectors.groupingBy(
                                 child -> !(child instanceof SlotReference),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java
index 61aac4d2407..edbd28677b4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java
@@ -70,6 +70,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate.PushDownAggOp;
+import org.apache.doris.nereids.types.TinyIntType;
 import org.apache.doris.nereids.util.ExpressionUtils;
 import org.apache.doris.nereids.util.TypeCoercionUtils;
 import org.apache.doris.qe.ConnectContext;
@@ -1292,6 +1293,15 @@ public class AggregateStrategies implements ImplementationRuleFactory {
                 .build();
 
         List<Expression> localAggGroupBy = ImmutableList.copyOf(localAggGroupBySet);
+        boolean isGroupByEmptySelectEmpty = localAggGroupBy.isEmpty() && localAggOutput.isEmpty();
+
+        // be not recommend generate an aggregate node with empty group by and empty output,
+        // so add a null int slot to group by slot and output
+        if (isGroupByEmptySelectEmpty) {
+            localAggGroupBy = ImmutableList.of(new NullLiteral(TinyIntType.INSTANCE));
+            localAggOutput = ImmutableList.of(new Alias(new NullLiteral(TinyIntType.INSTANCE)));
+        }
+
         boolean maybeUsingStreamAgg = maybeUsingStreamAgg(connectContext, localAggGroupBy);
         List<Expression> partitionExpressions = getHashAggregatePartitionExpressions(logicalAgg);
         RequireProperties requireAny = RequireProperties.of(PhysicalProperties.ANY);
@@ -1317,6 +1327,12 @@ public class AggregateStrategies implements ImplementationRuleFactory {
                 .addAll(nonDistinctAggFunctionToAliasPhase2.values())
                 .build();
 
+        // be not recommend generate an aggregate node with empty group by and empty output,
+        // so add a null int slot to group by slot and output
+        if (isGroupByEmptySelectEmpty) {
+            globalAggOutput = ImmutableList.of(new Alias(new NullLiteral(TinyIntType.INSTANCE)));
+        }
+
         RequireProperties requireGather = RequireProperties.of(PhysicalProperties.GATHER);
         PhysicalHashAggregate<Plan> anyLocalGatherGlobalAgg = new PhysicalHashAggregate<>(
                 localAggGroupBy, globalAggOutput, Optional.of(partitionExpressions),
@@ -1680,6 +1696,16 @@ public class AggregateStrategies implements ImplementationRuleFactory {
         boolean maybeUsingStreamAgg = maybeUsingStreamAgg(connectContext, localAggGroupBy);
         List<Expression> partitionExpressions = getHashAggregatePartitionExpressions(logicalAgg);
         RequireProperties requireAny = RequireProperties.of(PhysicalProperties.ANY);
+
+        boolean isGroupByEmptySelectEmpty = localAggGroupBy.isEmpty() && localAggOutput.isEmpty();
+
+        // be not recommend generate an aggregate node with empty group by and empty output,
+        // so add a null int slot to group by slot and output
+        if (isGroupByEmptySelectEmpty) {
+            localAggGroupBy = ImmutableList.of(new NullLiteral(TinyIntType.INSTANCE));
+            localAggOutput = ImmutableList.of(new Alias(new NullLiteral(TinyIntType.INSTANCE)));
+        }
+
         PhysicalHashAggregate<Plan> anyLocalAgg = new PhysicalHashAggregate<>(localAggGroupBy,
                 localAggOutput, Optional.of(partitionExpressions), inputToBufferParam,
                 maybeUsingStreamAgg, Optional.empty(), logicalAgg.getLogicalProperties(),
@@ -1702,6 +1728,12 @@ public class AggregateStrategies implements ImplementationRuleFactory {
                 .addAll(nonDistinctAggFunctionToAliasPhase2.values())
                 .build();
 
+        // be not recommend generate an aggregate node with empty group by and empty output,
+        // so add a null int slot to group by slot and output
+        if (isGroupByEmptySelectEmpty) {
+            globalAggOutput = ImmutableList.of(new Alias(new NullLiteral(TinyIntType.INSTANCE)));
+        }
+
         RequireProperties requireGather = RequireProperties.of(PhysicalProperties.GATHER);
 
         RequireProperties requireDistinctHash = RequireProperties.of(


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


(doris) 04/04: [fix](Nereids) fix bind group by int literal (#33117)

Posted by yi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yiguolei pushed a commit to branch 2.1-tmp
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 9bc7902e5a0a4d8f10be1cb1e7bedbbc68b9fda6
Author: 924060929 <92...@qq.com>
AuthorDate: Mon Apr 1 21:28:53 2024 +0800

    [fix](Nereids) fix bind group by int literal (#33117)
    
    This sql will failed because
    
        2 in the group by will bind to 1 as col2 in BindExpression
        ResolveOrdinalInOrderByAndGroupBy will replace 1 to MIN (LENGTH (cast(age as varchar)))
        CheckAnalysis will throw an exception because group by can not contains aggregate function
    
    select MIN (LENGTH (cast(age as varchar))), 1 AS col2
    from test_bind_groupby_slots
    group by 2
    
    we should move ResolveOrdinalInOrderByAndGroupBy into BindExpression
    
    (cherry picked from commit 3fab4496c3fefe95b4db01f300bf747080bfc3d8)
---
 .../doris/nereids/jobs/executor/Analyzer.java      |   2 -
 .../nereids/rules/analysis/BindExpression.java     |  33 ++++++-
 .../ResolveOrdinalInOrderByAndGroupBy.java         | 102 ---------------------
 .../data/nereids_syntax_p0/bind_priority.out       |   6 ++
 .../suites/nereids_syntax_p0/bind_priority.groovy  |  28 ++++++
 5 files changed, 63 insertions(+), 108 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java
index 9ad10a30aa2..a0431e066be 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java
@@ -43,7 +43,6 @@ import org.apache.doris.nereids.rules.analysis.OneRowRelationExtractAggregate;
 import org.apache.doris.nereids.rules.analysis.ProjectToGlobalAggregate;
 import org.apache.doris.nereids.rules.analysis.ProjectWithDistinctToAggregate;
 import org.apache.doris.nereids.rules.analysis.ReplaceExpressionByChildOutput;
-import org.apache.doris.nereids.rules.analysis.ResolveOrdinalInOrderByAndGroupBy;
 import org.apache.doris.nereids.rules.analysis.SubqueryToApply;
 import org.apache.doris.nereids.rules.rewrite.MergeProjects;
 import org.apache.doris.nereids.rules.rewrite.SemiJoinCommute;
@@ -147,7 +146,6 @@ public class Analyzer extends AbstractBatchJobExecutor {
                 // please see rule BindSlotReference or BindFunction for example
                 new EliminateDistinctConstant(),
                 new ProjectWithDistinctToAggregate(),
-                new ResolveOrdinalInOrderByAndGroupBy(),
                 new ReplaceExpressionByChildOutput(),
                 new OneRowRelationExtractAggregate()
             ),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java
index 6211f493eaf..43f89d5b010 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java
@@ -56,6 +56,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.GroupingScala
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Lambda;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.StructElement;
 import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction;
+import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.StringLiteral;
 import org.apache.doris.nereids.trees.plans.AbstractPlan;
 import org.apache.doris.nereids.trees.plans.JoinType;
@@ -486,11 +487,12 @@ public class BindExpression implements AnalysisRuleFactory {
         LogicalSort<LogicalSetOperation> sort = ctx.root;
         CascadesContext cascadesContext = ctx.cascadesContext;
 
+        List<Slot> childOutput = sort.child().getOutput();
         SimpleExprAnalyzer analyzer = buildSimpleExprAnalyzer(
                 sort, cascadesContext, sort.children(), true, true);
         Builder<OrderKey> boundKeys = ImmutableList.builderWithExpectedSize(sort.getOrderKeys().size());
         for (OrderKey orderKey : sort.getOrderKeys()) {
-            Expression boundKey = analyzer.analyze(orderKey.getExpr());
+            Expression boundKey = bindWithOrdinal(orderKey.getExpr(), analyzer, childOutput);
             boundKeys.add(orderKey.withExpression(boundKey));
         }
         return new LogicalSort<>(boundKeys.build(), sort.child());
@@ -699,7 +701,11 @@ public class BindExpression implements AnalysisRuleFactory {
                     return useOutputExpr.build();
                 });
 
-        List<Expression> boundGroupBy = analyzer.analyzeToList(groupBy);
+        ImmutableList.Builder<Expression> boundGroupByBuilder = ImmutableList.builderWithExpectedSize(groupBy.size());
+        for (Expression key : groupBy) {
+            boundGroupByBuilder.add(bindWithOrdinal(key, analyzer, boundAggOutput));
+        }
+        List<Expression> boundGroupBy = boundGroupByBuilder.build();
         checkIfOutputAliasNameDuplicatedForGroupBy(boundGroupBy, boundAggOutput);
         return boundGroupBy;
     }
@@ -723,6 +729,9 @@ public class BindExpression implements AnalysisRuleFactory {
     private Plan bindSortWithoutSetOperation(MatchingContext<LogicalSort<Plan>> ctx) {
         LogicalSort<Plan> sort = ctx.root;
         Plan input = sort.child();
+
+        List<Slot> childOutput = input.getOutput();
+
         // we should skip LogicalHaving to bind slot in LogicalSort;
         if (input instanceof LogicalHaving) {
             input = input.child(0);
@@ -744,7 +753,8 @@ public class BindExpression implements AnalysisRuleFactory {
         //        group by col1
         //        order by col1;     # order by order_col1
         //    bind order_col1 with alias_col1, then, bind it with inner_col1
-        Scope inputScope = toScope(cascadesContext, input.getOutput());
+        List<Slot> inputSlots = input.getOutput();
+        Scope inputScope = toScope(cascadesContext, inputSlots);
 
         final Plan finalInput = input;
         Supplier<Scope> inputChildrenScope = Suppliers.memoize(
@@ -766,7 +776,7 @@ public class BindExpression implements AnalysisRuleFactory {
 
         Builder<OrderKey> boundOrderKeys = ImmutableList.builderWithExpectedSize(sort.getOrderKeys().size());
         for (OrderKey orderKey : sort.getOrderKeys()) {
-            Expression boundKey = analyzer.analyze(orderKey.getExpr());
+            Expression boundKey = bindWithOrdinal(orderKey.getExpr(), analyzer, childOutput);
             boundOrderKeys.add(orderKey.withExpression(boundKey));
         }
         return new LogicalSort<>(boundOrderKeys.build(), sort.child());
@@ -858,6 +868,21 @@ public class BindExpression implements AnalysisRuleFactory {
                     unboundFunction.getDbName(), unboundFunction.getName());
     }
 
+    private Expression bindWithOrdinal(
+            Expression unbound, SimpleExprAnalyzer analyzer, List<? extends Expression> boundSelectOutput) {
+        if (unbound instanceof IntegerLikeLiteral) {
+            int ordinal = ((IntegerLikeLiteral) unbound).getIntValue();
+            if (ordinal >= 1 && ordinal <= boundSelectOutput.size()) {
+                Expression boundSelectItem = boundSelectOutput.get(ordinal - 1);
+                return boundSelectItem instanceof Alias ? boundSelectItem.child(0) : boundSelectItem;
+            } else {
+                return unbound; // bound literal
+            }
+        } else {
+            return analyzer.analyze(unbound);
+        }
+    }
+
     private <E extends Expression> E checkBoundExceptLambda(E expression, Plan plan) {
         if (expression instanceof Lambda) {
             return expression;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java
deleted file mode 100644
index 1cefd203ff7..00000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ResolveOrdinalInOrderByAndGroupBy.java
+++ /dev/null
@@ -1,102 +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.analysis;
-
-import org.apache.doris.nereids.properties.OrderKey;
-import org.apache.doris.nereids.rules.Rule;
-import org.apache.doris.nereids.rules.RuleType;
-import org.apache.doris.nereids.trees.expressions.Alias;
-import org.apache.doris.nereids.trees.expressions.Expression;
-import org.apache.doris.nereids.trees.expressions.NamedExpression;
-import org.apache.doris.nereids.trees.expressions.Slot;
-import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral;
-import org.apache.doris.nereids.trees.plans.Plan;
-import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
-import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
-
-import com.google.common.collect.ImmutableList;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * SELECT col1, col2 FROM t1 ORDER BY 1 -> SELECT col1, col2 FROM t1 ORDER BY col1
- * SELECT col1, SUM(col2) FROM t1 GROUP BY 1 -> SELECT col1, SUM(col2) FROM t1 GROUP BY col1
- */
-public class ResolveOrdinalInOrderByAndGroupBy implements AnalysisRuleFactory {
-
-    @Override
-    public List<Rule> buildRules() {
-        return ImmutableList.<Rule>builder()
-                .add(RuleType.RESOLVE_ORDINAL_IN_ORDER_BY.build(
-                        logicalSort().thenApply(ctx -> {
-                            LogicalSort<Plan> sort = ctx.root;
-                            List<Slot> childOutput = sort.child().getOutput();
-                            List<OrderKey> orderKeys = sort.getOrderKeys();
-                            List<OrderKey> orderKeysWithoutOrd = new ArrayList<>();
-                            for (OrderKey k : orderKeys) {
-                                Expression expression = k.getExpr();
-                                if (expression instanceof IntegerLikeLiteral) {
-                                    IntegerLikeLiteral i = (IntegerLikeLiteral) expression;
-                                    int ord = i.getIntValue();
-                                    checkOrd(ord, childOutput.size());
-                                    orderKeysWithoutOrd
-                                            .add(new OrderKey(childOutput.get(ord - 1), k.isAsc(), k.isNullFirst()));
-                                } else {
-                                    orderKeysWithoutOrd.add(k);
-                                }
-                            }
-                            return sort.withOrderKeys(orderKeysWithoutOrd);
-                        })
-                ))
-                .add(RuleType.RESOLVE_ORDINAL_IN_GROUP_BY.build(
-                        logicalAggregate().whenNot(LogicalAggregate::isOrdinalIsResolved).thenApply(ctx -> {
-                            LogicalAggregate<Plan> agg = ctx.root;
-                            List<NamedExpression> aggOutput = agg.getOutputExpressions();
-                            List<Expression> groupByWithoutOrd = new ArrayList<>();
-                            boolean ordExists = false;
-                            for (Expression groupByExpr : agg.getGroupByExpressions()) {
-                                if (groupByExpr instanceof IntegerLikeLiteral) {
-                                    IntegerLikeLiteral i = (IntegerLikeLiteral) groupByExpr;
-                                    int ord = i.getIntValue();
-                                    checkOrd(ord, aggOutput.size());
-                                    Expression aggExpr = aggOutput.get(ord - 1);
-                                    if (aggExpr instanceof Alias) {
-                                        aggExpr = ((Alias) aggExpr).child();
-                                    }
-                                    groupByWithoutOrd.add(aggExpr);
-                                    ordExists = true;
-                                } else {
-                                    groupByWithoutOrd.add(groupByExpr);
-                                }
-                            }
-                            if (ordExists) {
-                                return new LogicalAggregate<>(groupByWithoutOrd, agg.getOutputExpressions(),
-                                        true, agg.child());
-                            } else {
-                                return agg;
-                            }
-                        }))).build();
-    }
-
-    private void checkOrd(int ord, int childOutputSize) {
-        if (ord < 1 || ord > childOutputSize) {
-            throw new IllegalStateException(String.format("ordinal exceeds number of items in select list: %s", ord));
-        }
-    }
-}
diff --git a/regression-test/data/nereids_syntax_p0/bind_priority.out b/regression-test/data/nereids_syntax_p0/bind_priority.out
index 53432880c24..56706546bab 100644
--- a/regression-test/data/nereids_syntax_p0/bind_priority.out
+++ b/regression-test/data/nereids_syntax_p0/bind_priority.out
@@ -85,3 +85,9 @@ all	2
 -- !having_bind_group_by --
 2	1
 
+-- !sql --
+2	1
+
+-- !sql --
+2	1
+
diff --git a/regression-test/suites/nereids_syntax_p0/bind_priority.groovy b/regression-test/suites/nereids_syntax_p0/bind_priority.groovy
index 84bab14eba0..769f1771982 100644
--- a/regression-test/suites/nereids_syntax_p0/bind_priority.groovy
+++ b/regression-test/suites/nereids_syntax_p0/bind_priority.groovy
@@ -309,4 +309,32 @@ suite("bind_priority") {
                 having pk = 2;
                 """
     }()
+
+    def bindGroupBy = {
+        sql "drop table if exists test_bind_groupby_slots"
+
+        sql """create table test_bind_groupby_slots
+                (id int, age int)
+                distributed by hash(id)
+                properties('replication_num'='1');
+                """
+        sql "insert into test_bind_groupby_slots values(1, 10), (2, 20), (3, 30);"
+
+        order_qt_sql "select MIN (LENGTH (cast(age as varchar))), 1 AS col2 from test_bind_groupby_slots group by 2"
+    }()
+
+
+
+    def bindOrderBy = {
+        sql "drop table if exists test_bind_orderby_slots"
+
+        sql """create table test_bind_orderby_slots
+                (id int, age int)
+                distributed by hash(id)
+                properties('replication_num'='1');
+                """
+        sql "insert into test_bind_orderby_slots values(1, 10), (2, 20), (3, 30);"
+
+        order_qt_sql "select MIN (LENGTH (cast(age as varchar))), 1 AS col2 from test_bind_orderby_slots order by 2"
+    }()
 }


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


(doris) 01/04: [enhancement](Nereids) refactor expression rewriter to pattern match (#32617)

Posted by yi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yiguolei pushed a commit to branch 2.1-tmp
in repository https://gitbox.apache.org/repos/asf/doris.git

commit ff990eb869329eea4a6a22d59d83d6cec8edd5ee
Author: 924060929 <92...@qq.com>
AuthorDate: Wed Apr 10 13:30:00 2024 +0800

    [enhancement](Nereids) refactor expression rewriter to pattern match (#32617)
    
    this pr can improve the performance of the nereids planner, in plan stage.
    
    1. refactor expression rewriter to pattern match, so the lots of expression rewrite rules can criss-crossed apply in a big bottom-up iteration, and rewrite until the expression became stable. now we can process more cases because original there has no loop, and sometimes only process the top expression, like `SimplifyArithmeticRule`.
    2. replace `Collection.stream()` to `ImmutableXxx.Builder` to avoid useless method call
    3. loop unrolling some codes, like `Expression.<init>`, `PlanTreeRewriteBottomUpJob.pushChildrenJobs`
    4. use type/arity specified-code, like `OneRangePartitionEvaluator.toNereidsLiterals()`, `PartitionRangeExpander.tryExpandRange()`, `PartitionRangeExpander.enumerableCount()`
    5. refactor `ExtractCommonFactorRule`, now we can extract more cases, and I fix the deed loop when use `ExtractCommonFactorRule` and `SimplifyRange` in one iterative, because `SimplifyRange` generate right deep tree, but `ExtractCommonFactorRule` generate left deep tree
    6. refactor `FoldConstantRuleOnFE`, support visitor/pattern match mode, in ExpressionNormalization, pattern match can criss-crossed apply with other rules; in PartitionPruner, visitor can evaluate expression faster
    7. lazy compute and cache some operation
    8. use int field to compare date
    9. use BitSet to find disableNereidsRules
    10. two level loop usually faster then build Multimap when bind slot in Scope, so I revert the code
    11. `PlanTreeRewriteBottomUpJob` don't need to clearStatePhase any more
    
    ### test case
    100 threads parallel continuous send this sql which query an empty table, test in my mac machine(m2 chip, 8 core), enable sql cache
    ```sql
    select  count(1),date_format(time_col,'%Y%m%d'),varchar_col1
    from tbl
    where  partition_date>'2024-02-15'  and (varchar_col2 ='73130' or varchar_col3='73130') and time_col>'2024-03-04'
      and  time_col<'2024-03-05'
    group by date_format(time_col,'%Y%m%d'),varchar_col1
    order by date_format(time_col,'%Y%m%d') desc, varchar_col1 desc,count(1) asc
    limit 1000
    ```
    
    before this pr: 3100 peak QPS, about 2700 avg QPS
    after this pr: 4800 peak QPS, about 4400 avg QPS
    
    (cherry picked from commit 7338683fdbdf77711f2ce61e580c19f4ea100723)
---
 fe/fe-core/pom.xml                                 |   2 +-
 .../org/apache/doris/analysis/DateLiteral.java     |   9 +-
 .../java/org/apache/doris/catalog/OlapTable.java   |   5 +-
 .../org/apache/doris/mtmv/MTMVRelationManager.java |   4 +-
 .../org/apache/doris/mysql/privilege/Role.java     |   4 +-
 .../org/apache/doris/nereids/CascadesContext.java  |  49 +++-
 .../org/apache/doris/nereids/NereidsPlanner.java   |   2 +-
 .../org/apache/doris/nereids/StatementContext.java |  15 ++
 .../org/apache/doris/nereids/analyzer/Scope.java   |  19 +-
 .../java/org/apache/doris/nereids/jobs/Job.java    |  11 +-
 .../doris/nereids/jobs/executor/Rewriter.java      |   7 +-
 .../jobs/joinorder/hypergraph/HyperGraph.java      |  11 +-
 .../nereids/jobs/rewrite/CustomRewriteJob.java     |   6 +-
 .../jobs/rewrite/PlanTreeRewriteBottomUpJob.java   | 113 ++++++----
 .../nereids/jobs/rewrite/PlanTreeRewriteJob.java   |  64 ++++--
 .../jobs/rewrite/PlanTreeRewriteTopDownJob.java    |  41 +++-
 .../nereids/jobs/rewrite/RewriteJobContext.java    |  10 +-
 .../jobs/rewrite/RootPlanTreeRewriteJob.java       |  16 +-
 .../nereids/pattern/ExpressionPatternRules.java    | 112 +++++++++
 .../ExpressionPatternTraverseListeners.java        | 112 +++++++++
 .../doris/nereids/pattern/ParentTypeIdMapping.java |  59 +++++
 .../org/apache/doris/nereids/pattern/Pattern.java  |   4 +
 .../apache/doris/nereids/pattern/TypeMappings.java | 133 +++++++++++
 .../generator/ExpressionTypeMappingGenerator.java  | 159 +++++++++++++
 ...GeneratorAnalyzer.java => JavaAstAnalyzer.java} |  93 ++++----
 .../generator/LogicalBinaryPatternGenerator.java   |   4 +-
 .../generator/LogicalLeafPatternGenerator.java     |   4 +-
 .../generator/LogicalUnaryPatternGenerator.java    |   4 +-
 .../generator/PatternDescribableProcessor.java     |  34 ++-
 .../generator/PhysicalBinaryPatternGenerator.java  |   4 +-
 .../generator/PhysicalLeafPatternGenerator.java    |   4 +-
 .../generator/PhysicalUnaryPatternGenerator.java   |   4 +-
 ...ernGenerator.java => PlanPatternGenerator.java} |  18 +-
 .../generator/PlanPatternGeneratorAnalyzer.java    |  73 ++++++
 .../generator/PlanTypeMappingGenerator.java        | 159 +++++++++++++
 .../processor/post/RuntimeFilterPruner.java        |  17 +-
 .../doris/nereids/processor/post/Validator.java    |  10 +-
 .../nereids/properties/FunctionalDependencies.java |  24 +-
 .../nereids/properties/LogicalProperties.java      |  50 +++--
 .../java/org/apache/doris/nereids/rules/Rule.java  |   6 +-
 .../org/apache/doris/nereids/rules/RuleSet.java    |   4 +-
 .../AdjustAggregateNullableForEmptySet.java        |  29 ++-
 .../nereids/rules/analysis/BindExpression.java     |  28 ++-
 .../nereids/rules/analysis/BindSlotWithPaths.java  |  29 +--
 .../nereids/rules/analysis/CheckAfterRewrite.java  |  85 +++----
 .../nereids/rules/analysis/CheckAnalysis.java      |  36 +--
 .../rules/analysis/EliminateGroupByConstant.java   |   2 +-
 .../nereids/rules/analysis/ExpressionAnalyzer.java |   2 +-
 .../nereids/rules/analysis/FillUpMissingSlots.java |  21 +-
 .../nereids/rules/analysis/NormalizeAggregate.java |  41 ++--
 .../analysis/ReplaceExpressionByChildOutput.java   |  48 ++--
 .../nereids/rules/analysis/SubqueryToApply.java    |  77 +++++--
 .../mv/AbstractMaterializedViewRule.java           |  15 +-
 .../mv/InitMaterializationContextHook.java         |   4 +-
 .../exploration/mv/MaterializationContext.java     |   8 +-
 .../nereids/rules/exploration/mv/StructInfo.java   |  20 +-
 .../expression/ExpressionBottomUpRewriter.java     | 124 ++++++++++
 .../expression/ExpressionListenerMatcher.java      |  41 ++++
 ...eContext.java => ExpressionMatchingAction.java} |  16 +-
 ...Context.java => ExpressionMatchingContext.java} |  25 ++-
 .../rules/expression/ExpressionNormalization.java  |  29 +--
 ...=> ExpressionNormalizationAndOptimization.java} |  22 +-
 .../rules/expression/ExpressionOptimization.java   |  26 ++-
 .../expression/ExpressionPatternMatchRule.java     |  64 ++++++
 .../rules/expression/ExpressionPatternMatcher.java |  41 ++++
 .../expression/ExpressionPatternRuleFactory.java   |  84 +++++++
 .../rules/expression/ExpressionRewrite.java        |  51 ++++-
 .../rules/expression/ExpressionRewriteContext.java |   4 +-
 .../rules/expression/ExpressionRuleExecutor.java   |  16 +-
 ...ontext.java => ExpressionTraverseListener.java} |  16 +-
 .../ExpressionTraverseListenerFactory.java         |  79 +++++++
 .../ExpressionTraverseListenerMapping.java         |  59 +++++
 .../nereids/rules/expression/check/CheckCast.java  |  24 +-
 .../rules/ArrayContainToArrayOverlap.java          |  94 +++++---
 .../rules/expression/rules/CaseWhenToIf.java       |  18 +-
 .../expression/rules/ConvertAggStateCast.java      |  33 ++-
 .../expression/rules/DateFunctionRewrite.java      |  34 ++-
 .../expression/rules/DigitalMaskingConvert.java    |  23 +-
 .../expression/rules/DistinctPredicatesRule.java   |  18 +-
 .../expression/rules/ExtractCommonFactorRule.java  | 222 +++++++++++++++---
 .../rules/expression/rules/FoldConstantRule.java   |  32 ++-
 .../expression/rules/FoldConstantRuleOnBE.java     |  46 ++--
 .../expression/rules/FoldConstantRuleOnFE.java     | 170 ++++++++++++--
 .../rules/expression/rules/InPredicateDedup.java   |  40 ++--
 .../expression/rules/InPredicateToEqualToRule.java |  25 ++-
 .../rules/NormalizeBinaryPredicatesRule.java       |  21 +-
 .../expression/rules/NullSafeEqualToEqual.java     |  21 +-
 .../rules/OneListPartitionEvaluator.java           |   2 +-
 .../rules/OneRangePartitionEvaluator.java          | 120 ++++++----
 .../nereids/rules/expression/rules/OrToIn.java     |  36 +--
 .../rules/expression/rules/PartitionPruner.java    |  23 +-
 .../expression/rules/PartitionRangeExpander.java   | 115 ++++++----
 .../rules/PredicateRewriteForPartitionPrune.java   |   4 +-
 .../rules/RangePartitionValueIterator.java         |  64 ++++++
 .../expression/rules/ReplaceVariableByLiteral.java |  17 +-
 .../rules/SimplifyArithmeticComparisonRule.java    | 105 +++++----
 .../expression/rules/SimplifyArithmeticRule.java   |  70 +++---
 .../rules/expression/rules/SimplifyCastRule.java   |  21 +-
 .../rules/SimplifyComparisonPredicate.java         |  37 ++-
 .../rules/SimplifyDecimalV3Comparison.java         |  24 +-
 .../expression/rules/SimplifyInPredicate.java      |  20 +-
 .../expression/rules/SimplifyNotExprRule.java      |  34 ++-
 .../rules/expression/rules/SimplifyRange.java      |  73 +++---
 .../expression/rules/SupportJavaDateFormatter.java |  44 ++--
 .../nereids/rules/expression/rules/TopnToMax.java  |  29 ++-
 .../rules/TryEliminateUninterestedPredicates.java  |  14 +-
 .../rules/implementation/AggregateStrategies.java  |   2 +-
 .../rules/rewrite/AdjustConjunctsReturnType.java   |   4 +-
 .../nereids/rules/rewrite/AdjustNullable.java      |  12 +-
 .../rules/rewrite/CheckMatchExpression.java        |   7 +-
 .../nereids/rules/rewrite/CheckPrivileges.java     |  29 ++-
 .../doris/nereids/rules/rewrite/ColumnPruning.java |  98 ++++----
 .../rules/rewrite/CountDistinctRewrite.java        |  60 ++---
 .../nereids/rules/rewrite/CountLiteralRewrite.java |  37 ++-
 .../nereids/rules/rewrite/EliminateFilter.java     |   7 +-
 .../nereids/rules/rewrite/EliminateGroupBy.java    |  56 +++--
 .../nereids/rules/rewrite/EliminateMarkJoin.java   |  17 +-
 .../nereids/rules/rewrite/EliminateNotNull.java    |  39 ++--
 .../rules/rewrite/EliminateOrderByConstant.java    |  16 +-
 .../ExtractAndNormalizeWindowExpression.java       | 161 +++++++------
 ...xtractSingleTableExpressionFromDisjunction.java |   9 +-
 .../nereids/rules/rewrite/InferJoinNotNull.java    |   4 +-
 .../nereids/rules/rewrite/MergeAggregate.java      |   2 +-
 .../doris/nereids/rules/rewrite/MergeProjects.java |  10 +-
 .../doris/nereids/rules/rewrite/NormalizeSort.java |  59 +++--
 .../nereids/rules/rewrite/NormalizeToSlot.java     |  43 ++--
 .../rules/rewrite/PruneOlapScanPartition.java      |  51 +++--
 .../nereids/rules/rewrite/PullUpPredicates.java    |  75 ++++---
 .../rewrite/PushDownFilterThroughAggregation.java  |  12 +-
 .../rewrite/PushDownFilterThroughProject.java      |  13 +-
 .../doris/nereids/rules/rewrite/ReorderJoin.java   |   6 +-
 .../nereids/rules/rewrite/SimplifyAggGroupBy.java  |  23 +-
 .../mv/AbstractSelectMaterializedIndexRule.java    |  15 +-
 .../mv/SelectMaterializedIndexWithAggregate.java   |   9 +-
 .../SelectMaterializedIndexWithoutAggregate.java   |  45 ++--
 .../doris/nereids/stats/StatsCalculator.java       |  11 +-
 .../doris/nereids/trees/AbstractTreeNode.java      |  22 +-
 .../org/apache/doris/nereids/trees/TreeNode.java   |  17 ++
 .../nereids/trees/expressions/BinaryOperator.java  |   6 -
 .../trees/expressions/ComparisonPredicate.java     |   4 +-
 .../nereids/trees/expressions/Expression.java      |  99 +++++---
 .../nereids/trees/expressions/InPredicate.java     |   5 +-
 .../nereids/trees/expressions/SlotReference.java   |   7 +-
 .../functions/ComputeSignatureHelper.java          |  11 +-
 .../functions/agg/AggregateFunction.java           |  17 +-
 .../scalar/PushDownToProjectionFunction.java       |   7 +-
 .../trees/expressions/literal/DateLiteral.java     |  39 +++-
 .../visitor/DefaultExpressionRewriter.java         |  10 +-
 .../doris/nereids/trees/plans/AbstractPlan.java    |  28 +--
 .../org/apache/doris/nereids/trees/plans/Plan.java |  61 +++--
 .../nereids/trees/plans/algebra/Aggregate.java     |  17 +-
 .../doris/nereids/trees/plans/algebra/Project.java |  27 ++-
 .../trees/plans/logical/LogicalAggregate.java      |   8 +-
 .../plans/logical/LogicalCatalogRelation.java      | 132 ++++++-----
 .../trees/plans/logical/LogicalOlapScan.java       |  65 ++++--
 .../trees/plans/logical/LogicalProject.java        |   8 +-
 .../nereids/trees/plans/logical/LogicalSort.java   |  19 +-
 .../nereids/trees/plans/logical/LogicalTopN.java   |  13 +-
 .../trees/plans/physical/PhysicalHashJoin.java     |   3 +-
 .../apache/doris/nereids/util/ExpressionUtils.java | 250 ++++++++++++++++-----
 .../doris/nereids/util/ImmutableEqualSet.java      |   6 +-
 .../org/apache/doris/nereids/util/JoinUtils.java   |   5 +-
 .../org/apache/doris/nereids/util/PlanUtils.java   |  24 ++
 .../doris/nereids/util/TypeCoercionUtils.java      |  19 +-
 .../java/org/apache/doris/nereids/util/Utils.java  |  65 +++++-
 .../java/org/apache/doris/qe/SessionVariable.java  |  38 +++-
 .../jobs/joinorder/hypergraph/HyperGraphTest.java  |  12 +-
 .../rules/expression/ExpressionRewriteTest.java    |  80 +++++--
 .../expression/ExpressionRewriteTestHelper.java    |   2 +-
 .../nereids/rules/expression/FoldConstantTest.java |  36 ++-
 .../rules/expression/PredicatesSplitterTest.java   |   2 +-
 .../expression/SimplifyArithmeticRuleTest.java     |  56 +++--
 .../rules/expression/SimplifyInPredicateTest.java  |   8 +-
 .../rules/expression/SimplifyRangeTest.java        |  26 ++-
 .../expression/rules/NullSafeEqualToEqualTest.java |  20 +-
 .../SimplifyArithmeticComparisonRuleTest.java      |   7 +-
 .../expression/rules/SimplifyCastRuleTest.java     |   7 +-
 .../rules/SimplifyComparisonPredicateTest.java     |  35 ++-
 .../rules/SimplifyDecimalV3ComparisonTest.java     |   6 +-
 .../rules/expression/rules/TopnToMaxTest.java      |   4 +-
 .../rules/rewrite/EliminateJoinByFkTest.java       |   1 +
 .../doris/nereids/rules/rewrite/OrToInTest.java    |  19 +-
 .../PushDownFilterThroughAggregationTest.java      |   4 +-
 .../functions/ComputeSignatureHelperTest.java      |  11 +
 .../org/apache/doris/nereids/util/PlanChecker.java |  20 ++
 .../data/nereids_hint_tpcds_p0/shape/query24.out   |  10 +-
 .../data/nereids_hint_tpcds_p0/shape/query64.out   |   2 +-
 .../filter_push_down/push_filter_through.out       |  28 +--
 .../shape/query13.out                              |   2 +-
 .../shape/query14.out                              |   2 +-
 .../shape/query24.out                              |   4 +-
 .../shape/query41.out                              |   2 +-
 .../shape/query50.out                              |   2 +-
 .../shape/query64.out                              |   2 +-
 .../shape/query85.out                              |   2 +-
 .../shape/query95.out                              |   2 +-
 .../noStatsRfPrune/query13.out                     |   2 +-
 .../noStatsRfPrune/query17.out                     |   2 +-
 .../noStatsRfPrune/query41.out                     |   2 +-
 .../noStatsRfPrune/query47.out                     |   7 +-
 .../noStatsRfPrune/query50.out                     |   2 +-
 .../noStatsRfPrune/query57.out                     |   7 +-
 .../noStatsRfPrune/query6.out                      |  57 +++--
 .../noStatsRfPrune/query65.out                     |   2 +-
 .../no_stats_shape/query13.out                     |   2 +-
 .../no_stats_shape/query14.out                     |   2 +-
 .../no_stats_shape/query17.out                     |   2 +-
 .../no_stats_shape/query24.out                     |   2 +-
 .../no_stats_shape/query41.out                     |   2 +-
 .../no_stats_shape/query47.out                     |   7 +-
 .../no_stats_shape/query50.out                     |   2 +-
 .../no_stats_shape/query57.out                     |   7 +-
 .../no_stats_shape/query6.out                      |  57 +++--
 .../no_stats_shape/query64.out                     |   2 +-
 .../no_stats_shape/query65.out                     |   2 +-
 .../no_stats_shape/query85.out                     |   6 +-
 .../rf_prune/query13.out                           |   2 +-
 .../rf_prune/query14.out                           |   2 +-
 .../rf_prune/query41.out                           |   2 +-
 .../rf_prune/query50.out                           |   2 +-
 .../rf_prune/query85.out                           |   2 +-
 .../rf_prune/query95.out                           |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query13.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query14.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query24.out |   4 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query41.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query50.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query64.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query85.out |   2 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query95.out |   2 +-
 .../data/nereids_tpch_shape_sf1000_p0/shape/q9.out |   2 +-
 .../shape_no_stats/q9.out                          |   2 +-
 .../org/apache/doris/regression/suite/Suite.groovy |   7 +-
 .../doris/regression/util/OutputUtils.groovy       |  28 ++-
 .../doris/regression/util/ReusableIterator.groovy  |   7 +
 235 files changed, 4959 insertions(+), 1963 deletions(-)

diff --git a/fe/fe-core/pom.xml b/fe/fe-core/pom.xml
index dae3740eb71..83deec7650c 100644
--- a/fe/fe-core/pom.xml
+++ b/fe/fe-core/pom.xml
@@ -1016,7 +1016,7 @@ under the License.
                         <configuration>
                             <proc>only</proc>
                             <compilerArgs>
-                                <arg>-AplanPath=${basedir}/src/main/java/org/apache/doris/nereids</arg>
+                                <arg>-Apath=${basedir}/src/main/java/org/apache/doris/nereids</arg>
                             </compilerArgs>
                             <includes>
                                 <include>org/apache/doris/nereids/pattern/generator/PatternDescribableProcessPoint.java</include>
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
index 28ed98df0cb..125f1a56c9c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
@@ -570,11 +570,14 @@ public class DateLiteral extends LiteralExpr {
         switch (type.getPrimitiveType()) {
             case DATE:
             case DATEV2:
-                return this.getStringValue().compareTo(MIN_DATE.getStringValue()) == 0;
+                return year == 0 && month == 1 && day == 1
+                        && this.getStringValue().compareTo(MIN_DATE.getStringValue()) == 0;
             case DATETIME:
-                return this.getStringValue().compareTo(MIN_DATETIME.getStringValue()) == 0;
+                return year == 0 && month == 1 && day == 1
+                        && this.getStringValue().compareTo(MIN_DATETIME.getStringValue()) == 0;
             case DATETIMEV2:
-                return this.getStringValue().compareTo(MIN_DATETIMEV2.getStringValue()) == 0;
+                return year == 0 && month == 1 && day == 1
+                        && this.getStringValue().compareTo(MIN_DATETIMEV2.getStringValue()) == 0;
             default:
                 return false;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
index 459e5f360af..009b4c3b362 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
@@ -2454,9 +2454,8 @@ public class OlapTable extends Table implements MTMVRelatedTableIf {
     }
 
     public boolean isDupKeysOrMergeOnWrite() {
-        return getKeysType() == KeysType.DUP_KEYS
-                || (getKeysType() == KeysType.UNIQUE_KEYS
-                && getEnableUniqueKeyMergeOnWrite());
+        return keysType == KeysType.DUP_KEYS
+                || (keysType == KeysType.UNIQUE_KEYS && getEnableUniqueKeyMergeOnWrite());
     }
 
     public void initAutoIncrementGenerator(long dbId) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java
index 723deaff740..693bb4b19de 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java
@@ -67,7 +67,7 @@ public class MTMVRelationManager implements MTMVHookService {
      * @return
      */
     public Set<MTMV> getAvailableMTMVs(List<BaseTableInfo> tableInfos, ConnectContext ctx) {
-        Set<MTMV> res = Sets.newHashSet();
+        Set<MTMV> res = Sets.newLinkedHashSet();
         Set<BaseTableInfo> mvInfos = getMTMVInfos(tableInfos);
         for (BaseTableInfo tableInfo : mvInfos) {
             try {
@@ -90,7 +90,7 @@ public class MTMVRelationManager implements MTMVHookService {
     }
 
     private Set<BaseTableInfo> getMTMVInfos(List<BaseTableInfo> tableInfos) {
-        Set<BaseTableInfo> mvInfos = Sets.newHashSet();
+        Set<BaseTableInfo> mvInfos = Sets.newLinkedHashSet();
         for (BaseTableInfo tableInfo : tableInfos) {
             mvInfos.addAll(getMtmvsByBaseTable(tableInfo));
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java
index f59cfd699f0..8724331eb0f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Role.java
@@ -380,7 +380,9 @@ public class Role implements Writable, GsonPostProcessable {
 
     public boolean checkColPriv(String ctl, String db, String tbl, String col, PrivPredicate wanted) {
         Optional<Privilege> colPrivilege = wanted.getColPrivilege();
-        Preconditions.checkState(colPrivilege.isPresent(), "this privPredicate should not use checkColPriv:" + wanted);
+        if (!colPrivilege.isPresent()) {
+            throw new IllegalStateException("this privPredicate should not use checkColPriv:" + wanted);
+        }
         return checkTblPriv(ctl, db, tbl, wanted) || onlyCheckColPriv(ctl, db, tbl, col, colPrivilege.get());
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java
index 8e4a47938e4..60b7c0343a7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java
@@ -76,6 +76,7 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -134,6 +135,11 @@ public class CascadesContext implements ScheduleContext {
     // trigger by rule and show by `explain plan process` statement
     private final List<PlanProcess> planProcesses = new ArrayList<>();
 
+    // this field is modified by FoldConstantRuleOnFE, it matters current traverse
+    // into AggregateFunction with distinct, we can not fold constant in this case
+    private int distinctAggLevel;
+    private final boolean isEnableExprTrace;
+
     /**
      * Constructor of OptimizerContext.
      *
@@ -156,6 +162,13 @@ public class CascadesContext implements ScheduleContext {
         this.subqueryExprIsAnalyzed = new HashMap<>();
         this.runtimeFilterContext = new RuntimeFilterContext(getConnectContext().getSessionVariable());
         this.materializationContexts = new ArrayList<>();
+        if (statementContext.getConnectContext() != null) {
+            ConnectContext connectContext = statementContext.getConnectContext();
+            SessionVariable sessionVariable = connectContext.getSessionVariable();
+            this.isEnableExprTrace = sessionVariable != null && sessionVariable.isEnableExprTrace();
+        } else {
+            this.isEnableExprTrace = false;
+        }
     }
 
     /**
@@ -256,7 +269,7 @@ public class CascadesContext implements ScheduleContext {
         this.tables = tables.stream().collect(Collectors.toMap(TableIf::getId, t -> t, (t1, t2) -> t1));
     }
 
-    public ConnectContext getConnectContext() {
+    public final ConnectContext getConnectContext() {
         return statementContext.getConnectContext();
     }
 
@@ -366,12 +379,18 @@ public class CascadesContext implements ScheduleContext {
             return defaultValue;
         }
 
+        return getStatementContext().getOrRegisterCache(cacheName,
+                () -> variableSupplier.apply(connectContext.getSessionVariable()));
+    }
+
+    /** getAndCacheDisableRules */
+    public final BitSet getAndCacheDisableRules() {
+        ConnectContext connectContext = getConnectContext();
         StatementContext statementContext = getStatementContext();
-        if (statementContext == null) {
-            return defaultValue;
+        if (connectContext == null || statementContext == null) {
+            return new BitSet();
         }
-        return statementContext.getOrRegisterCache(cacheName,
-                () -> variableSupplier.apply(connectContext.getSessionVariable()));
+        return statementContext.getOrCacheDisableRules(connectContext.getSessionVariable());
     }
 
     private CascadesContext execute(Job job) {
@@ -718,8 +737,28 @@ public class CascadesContext implements ScheduleContext {
     }
 
     public void printPlanProcess() {
+        printPlanProcess(this.planProcesses);
+    }
+
+    public static void printPlanProcess(List<PlanProcess> planProcesses) {
         for (PlanProcess row : planProcesses) {
             LOG.info("RULE: " + row.ruleName + "\nBEFORE:\n" + row.beforeShape + "\nafter:\n" + row.afterShape);
         }
     }
+
+    public void incrementDistinctAggLevel() {
+        this.distinctAggLevel++;
+    }
+
+    public void decrementDistinctAggLevel() {
+        this.distinctAggLevel--;
+    }
+
+    public int getDistinctAggLevel() {
+        return distinctAggLevel;
+    }
+
+    public boolean isEnableExprTrace() {
+        return isEnableExprTrace;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java
index eedc77e9df7..7457e4de04a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java
@@ -387,7 +387,7 @@ public class NereidsPlanner extends Planner {
             if (hint instanceof DistributeHint) {
                 distributeHintIndex++;
                 if (!hint.getExplainString().equals("")) {
-                    distributeIndex = "_" + String.valueOf(distributeHintIndex);
+                    distributeIndex = "_" + distributeHintIndex;
                 }
             }
             switch (hint.getStatus()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java
index 5c894fd46ef..7b444995120 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java
@@ -36,6 +36,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.qe.SessionVariable;
 
 import com.google.common.base.Stopwatch;
 import com.google.common.base.Supplier;
@@ -45,6 +46,7 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -117,6 +119,8 @@ public class StatementContext {
     // Relation for example LogicalOlapScan
     private final Map<Slot, Relation> slotToRelation = Maps.newHashMap();
 
+    private BitSet disableRules;
+
     public StatementContext() {
         this.connectContext = ConnectContext.get();
     }
@@ -259,11 +263,22 @@ public class StatementContext {
         return supplier.get();
     }
 
+    public synchronized BitSet getOrCacheDisableRules(SessionVariable sessionVariable) {
+        if (this.disableRules != null) {
+            return this.disableRules;
+        }
+        this.disableRules = sessionVariable.getDisableNereidsRules();
+        return this.disableRules;
+    }
+
     /**
      * Some value of the cacheKey may change, invalid cache when value change
      */
     public synchronized void invalidCache(String cacheKey) {
         contextCacheMap.remove(cacheKey);
+        if (cacheKey.equalsIgnoreCase(SessionVariable.DISABLE_NEREIDS_RULES)) {
+            this.disableRules = null;
+        }
     }
 
     public ColumnAliasGenerator getColumnAliasGenerator() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/Scope.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/Scope.java
index a95e562f7e0..dbcbea7c104 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/Scope.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/Scope.java
@@ -26,6 +26,7 @@ import com.google.common.collect.LinkedListMultimap;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Sets;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.Locale;
 import java.util.Objects;
@@ -63,6 +64,7 @@ public class Scope {
     private final List<Slot> slots;
     private final Optional<SubqueryExpr> ownerSubquery;
     private final Set<Slot> correlatedSlots;
+    private final boolean buildNameToSlot;
     private final Supplier<ListMultimap<String, Slot>> nameToSlot;
 
     public Scope(List<? extends Slot> slots) {
@@ -75,7 +77,8 @@ public class Scope {
         this.slots = Utils.fastToImmutableList(Objects.requireNonNull(slots, "slots can not be null"));
         this.ownerSubquery = Objects.requireNonNull(subqueryExpr, "subqueryExpr can not be null");
         this.correlatedSlots = Sets.newLinkedHashSet();
-        this.nameToSlot = Suppliers.memoize(this::buildNameToSlot);
+        this.buildNameToSlot = slots.size() > 500;
+        this.nameToSlot = buildNameToSlot ? Suppliers.memoize(this::buildNameToSlot) : null;
     }
 
     public List<Slot> getSlots() {
@@ -96,7 +99,19 @@ public class Scope {
 
     /** findSlotIgnoreCase */
     public List<Slot> findSlotIgnoreCase(String slotName) {
-        return nameToSlot.get().get(slotName.toUpperCase(Locale.ROOT));
+        if (!buildNameToSlot) {
+            Object[] array = new Object[slots.size()];
+            int filterIndex = 0;
+            for (int i = 0; i < slots.size(); i++) {
+                Slot slot = slots.get(i);
+                if (slot.getName().equalsIgnoreCase(slotName)) {
+                    array[filterIndex++] = slot;
+                }
+            }
+            return (List) Arrays.asList(array).subList(0, filterIndex);
+        } else {
+            return nameToSlot.get().get(slotName.toUpperCase(Locale.ROOT));
+        }
     }
 
     private ListMultimap<String, Slot> buildNameToSlot() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/Job.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/Job.java
index a9739cbb9e2..41e5e1b8d7e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/Job.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/Job.java
@@ -34,16 +34,14 @@ import org.apache.doris.nereids.rules.RuleSet;
 import org.apache.doris.nereids.trees.expressions.CTEId;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.qe.SessionVariable;
 import org.apache.doris.statistics.Statistics;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableSet;
 
+import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Set;
 
 /**
  * Abstract class for all job using for analyze and optimize query plan in Nereids.
@@ -57,7 +55,7 @@ public abstract class Job implements TracerSupplier {
     protected JobType type;
     protected JobContext context;
     protected boolean once;
-    protected final Set<Integer> disableRules;
+    protected final BitSet disableRules;
 
     protected Map<CTEId, Statistics> cteIdToStats;
 
@@ -129,8 +127,7 @@ public abstract class Job implements TracerSupplier {
                 groupExpression.getOwnerGroup(), groupExpression, groupExpression.getPlan()));
     }
 
-    public static Set<Integer> getDisableRules(JobContext context) {
-        return context.getCascadesContext().getAndCacheSessionVariable(
-                SessionVariable.DISABLE_NEREIDS_RULES, ImmutableSet.of(), SessionVariable::getDisableNereidsRules);
+    public static BitSet getDisableRules(JobContext context) {
+        return context.getCascadesContext().getAndCacheDisableRules();
     }
 }
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 65998416fb0..a68c7510965 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
@@ -30,7 +30,7 @@ import org.apache.doris.nereids.rules.analysis.LogicalSubQueryAliasToLogicalProj
 import org.apache.doris.nereids.rules.analysis.NormalizeAggregate;
 import org.apache.doris.nereids.rules.expression.CheckLegalityAfterRewrite;
 import org.apache.doris.nereids.rules.expression.ExpressionNormalization;
-import org.apache.doris.nereids.rules.expression.ExpressionOptimization;
+import org.apache.doris.nereids.rules.expression.ExpressionNormalizationAndOptimization;
 import org.apache.doris.nereids.rules.expression.ExpressionRewrite;
 import org.apache.doris.nereids.rules.rewrite.AddDefaultLimit;
 import org.apache.doris.nereids.rules.rewrite.AdjustConjunctsReturnType;
@@ -152,8 +152,7 @@ public class Rewriter extends AbstractBatchJobExecutor {
                             //   such as group by key matching and replaced
                             //   but we need to do some normalization before subquery unnesting,
                             //   such as extract common expression.
-                            new ExpressionNormalization(),
-                            new ExpressionOptimization(),
+                            new ExpressionNormalizationAndOptimization(),
                             new AvgDistinctToSumDivCount(),
                             new CountDistinctRewrite(),
                             new ExtractFilterFromCrossJoin()
@@ -240,7 +239,7 @@ public class Rewriter extends AbstractBatchJobExecutor {
                     // efficient because it can find the new plans and apply transform wherever it is
                     bottomUp(RuleSet.PUSH_DOWN_FILTERS),
                     // after push down, some new filters are generated, which needs to be optimized. (example: tpch q19)
-                    topDown(new ExpressionOptimization()),
+                    // topDown(new ExpressionOptimization()),
                     topDown(
                             new MergeFilters(),
                             new ReorderJoin(),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperGraph.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperGraph.java
index 3b3c2f410c4..5e45fc0bdb8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperGraph.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperGraph.java
@@ -51,6 +51,7 @@ import com.google.common.collect.Sets;
 import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -64,7 +65,7 @@ import javax.annotation.Nullable;
  */
 public class HyperGraph {
     // record all edges that can be placed on the subgraph
-    private final Map<Long, BitSet> treeEdgesCache = new HashMap<>();
+    private final Map<Long, BitSet> treeEdgesCache = new LinkedHashMap<>();
     private final List<JoinEdge> joinEdges;
     private final List<FilterEdge> filterEdges;
     private final List<AbstractNode> nodes;
@@ -330,9 +331,9 @@ public class HyperGraph {
         private final List<AbstractNode> nodes = new ArrayList<>();
 
         // These hyperGraphs should be replaced nodes when building all
-        private final Map<Long, List<HyperGraph>> replacedHyperGraphs = new HashMap<>();
-        private final HashMap<Slot, Long> slotToNodeMap = new HashMap<>();
-        private final Map<Long, List<NamedExpression>> complexProject = new HashMap<>();
+        private final Map<Long, List<HyperGraph>> replacedHyperGraphs = new LinkedHashMap<>();
+        private final HashMap<Slot, Long> slotToNodeMap = new LinkedHashMap<>();
+        private final Map<Long, List<NamedExpression>> complexProject = new LinkedHashMap<>();
         private Set<Slot> finalOutputs;
 
         public List<AbstractNode> getNodes() {
@@ -522,7 +523,7 @@ public class HyperGraph {
          */
         private BitSet addJoin(LogicalJoin<?, ?> join,
                 Pair<BitSet, Long> leftEdgeNodes, Pair<BitSet, Long> rightEdgeNodes) {
-            HashMap<Pair<Long, Long>, Pair<List<Expression>, List<Expression>>> conjuncts = new HashMap<>();
+            Map<Pair<Long, Long>, Pair<List<Expression>, List<Expression>>> conjuncts = new LinkedHashMap<>();
             for (Expression expression : join.getHashJoinConjuncts()) {
                 // TODO: avoid calling calculateEnds if calNodeMap's results are same
                 Pair<Long, Long> ends = calculateEnds(calNodeMap(expression.getInputSlots()), leftEdgeNodes,
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CustomRewriteJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CustomRewriteJob.java
index 35e04b9f33f..0e58f1bc976 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CustomRewriteJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CustomRewriteJob.java
@@ -25,8 +25,8 @@ import org.apache.doris.nereids.rules.RuleType;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter;
 
+import java.util.BitSet;
 import java.util.Objects;
-import java.util.Set;
 import java.util.function.Supplier;
 
 /**
@@ -50,8 +50,8 @@ public class CustomRewriteJob implements RewriteJob {
 
     @Override
     public void execute(JobContext context) {
-        Set<Integer> disableRules = Job.getDisableRules(context);
-        if (disableRules.contains(ruleType.type())) {
+        BitSet disableRules = Job.getDisableRules(context);
+        if (disableRules.get(ruleType.type())) {
             return;
         }
         CascadesContext cascadesContext = context.getCascadesContext();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java
index 4f623e54500..60555a9cc04 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java
@@ -39,9 +39,9 @@ public class PlanTreeRewriteBottomUpJob extends PlanTreeRewriteJob {
     // Different 'RewriteState' has different actions,
     // so we will do specified action for each node based on their 'RewriteState'.
     private static final String REWRITE_STATE_KEY = "rewrite_state";
-
     private final RewriteJobContext rewriteJobContext;
     private final List<Rule> rules;
+    private final int batchId;
 
     enum RewriteState {
         // 'REWRITE_THIS' means the current plan node can be handled immediately. If the plan state is 'REWRITE_THIS',
@@ -59,22 +59,15 @@ public class PlanTreeRewriteBottomUpJob extends PlanTreeRewriteJob {
         super(JobType.BOTTOM_UP_REWRITE, context);
         this.rewriteJobContext = Objects.requireNonNull(rewriteJobContext, "rewriteContext cannot be null");
         this.rules = Objects.requireNonNull(rules, "rules cannot be null");
+        this.batchId = rewriteJobContext.batchId;
     }
 
     @Override
     public void execute() {
-        // For the bottom-up rewrite job, we need to reset the state of its children
-        // if the plan has changed after the rewrite. So we use the 'childrenVisited' to check this situation.
-        boolean clearStatePhase = !rewriteJobContext.childrenVisited;
-        if (clearStatePhase) {
-            traverseClearState();
-            return;
-        }
-
         // We'll do different actions based on their different states.
         // You can check the comment in 'RewriteState' structure for more details.
         Plan plan = rewriteJobContext.plan;
-        RewriteState state = getState(plan);
+        RewriteState state = getState(plan, batchId);
         switch (state) {
             case REWRITE_THIS:
                 rewriteThis();
@@ -90,33 +83,13 @@ public class PlanTreeRewriteBottomUpJob extends PlanTreeRewriteJob {
         }
     }
 
-    private void traverseClearState() {
-        // Reset the state for current node.
-        RewriteJobContext clearedStateContext = rewriteJobContext.withChildrenVisited(true);
-        setState(clearedStateContext.plan, RewriteState.REWRITE_THIS);
-        pushJob(new PlanTreeRewriteBottomUpJob(clearedStateContext, context, rules));
-
-        // Generate the new rewrite job for its children. Because the character of stack is 'first in, last out',
-        // so we can traverse reset the state for the plan node until the leaf node.
-        List<Plan> children = clearedStateContext.plan.children();
-        for (int i = children.size() - 1; i >= 0; i--) {
-            Plan child = children.get(i);
-            RewriteJobContext childRewriteJobContext = new RewriteJobContext(
-                    child, clearedStateContext, i, false);
-            // NOTICE: this relay on pull up cte anchor
-            if (!(rewriteJobContext.plan instanceof LogicalCTEAnchor)) {
-                pushJob(new PlanTreeRewriteBottomUpJob(childRewriteJobContext, context, rules));
-            }
-        }
-    }
-
     private void rewriteThis() {
         // Link the current node with the sub-plan to get the current plan which is used in the rewrite phase later.
         Plan plan = linkChildren(rewriteJobContext.plan, rewriteJobContext.childrenContext);
         RewriteResult rewriteResult = rewrite(plan, rules, rewriteJobContext);
         if (rewriteResult.hasNewPlan) {
             RewriteJobContext newJobContext = rewriteJobContext.withPlan(rewriteResult.plan);
-            RewriteState state = getState(rewriteResult.plan);
+            RewriteState state = getState(rewriteResult.plan, batchId);
             // Some eliminate rule will return a rewritten plan, for example the current node is eliminated
             // and return the child plan. So we don't need to handle it again.
             if (state == RewriteState.REWRITTEN) {
@@ -125,40 +98,82 @@ public class PlanTreeRewriteBottomUpJob extends PlanTreeRewriteJob {
             }
             // After the rewrite take effect, we should handle the children part again.
             pushJob(new PlanTreeRewriteBottomUpJob(newJobContext, context, rules));
-            setState(rewriteResult.plan, RewriteState.ENSURE_CHILDREN_REWRITTEN);
+            setState(rewriteResult.plan, RewriteState.ENSURE_CHILDREN_REWRITTEN, batchId);
         } else {
             // No new plan is generated, so just set the state of the current plan to 'REWRITTEN'.
-            setState(rewriteResult.plan, RewriteState.REWRITTEN);
+            setState(rewriteResult.plan, RewriteState.REWRITTEN, batchId);
             rewriteJobContext.setResult(rewriteResult.plan);
         }
     }
 
     private void ensureChildrenRewritten() {
-        // Similar to the function 'traverseClearState'.
         Plan plan = rewriteJobContext.plan;
-        setState(plan, RewriteState.REWRITE_THIS);
+        int batchId = rewriteJobContext.batchId;
+        setState(plan, RewriteState.REWRITE_THIS, batchId);
         pushJob(new PlanTreeRewriteBottomUpJob(rewriteJobContext, context, rules));
 
+        // some rule return new plan tree, which the number of new plan node > 1,
+        // we should transform this new plan nodes too.
+        // NOTICE: this relay on pull up cte anchor
+        if (!(rewriteJobContext.plan instanceof LogicalCTEAnchor)) {
+            pushChildrenJobs(plan);
+        }
+    }
+
+    private void pushChildrenJobs(Plan plan) {
         List<Plan> children = plan.children();
-        for (int i = children.size() - 1; i >= 0; i--) {
-            Plan child = children.get(i);
-            // some rule return new plan tree, which the number of new plan node > 1,
-            // we should transform this new plan nodes too.
-            RewriteJobContext childRewriteJobContext = new RewriteJobContext(
-                    child, rewriteJobContext, i, false);
-            // NOTICE: this relay on pull up cte anchor
-            if (!(rewriteJobContext.plan instanceof LogicalCTEAnchor)) {
+        switch (children.size()) {
+            case 0: return;
+            case 1:
+                Plan child = children.get(0);
+                RewriteJobContext childRewriteJobContext = new RewriteJobContext(
+                        child, rewriteJobContext, 0, false, batchId);
                 pushJob(new PlanTreeRewriteBottomUpJob(childRewriteJobContext, context, rules));
-            }
+                return;
+            case 2:
+                Plan right = children.get(1);
+                RewriteJobContext rightRewriteJobContext = new RewriteJobContext(
+                        right, rewriteJobContext, 1, false, batchId);
+                pushJob(new PlanTreeRewriteBottomUpJob(rightRewriteJobContext, context, rules));
+
+                Plan left = children.get(0);
+                RewriteJobContext leftRewriteJobContext = new RewriteJobContext(
+                        left, rewriteJobContext, 0, false, batchId);
+                pushJob(new PlanTreeRewriteBottomUpJob(leftRewriteJobContext, context, rules));
+                return;
+            default:
+                for (int i = children.size() - 1; i >= 0; i--) {
+                    child = children.get(i);
+                    childRewriteJobContext = new RewriteJobContext(
+                            child, rewriteJobContext, i, false, batchId);
+                    pushJob(new PlanTreeRewriteBottomUpJob(childRewriteJobContext, context, rules));
+                }
+        }
+    }
+
+    private static RewriteState getState(Plan plan, int currentBatchId) {
+        Optional<RewriteStateContext> state = plan.getMutableState(REWRITE_STATE_KEY);
+        if (!state.isPresent()) {
+            return RewriteState.ENSURE_CHILDREN_REWRITTEN;
+        }
+        RewriteStateContext context = state.get();
+        if (context.batchId != currentBatchId) {
+            return RewriteState.ENSURE_CHILDREN_REWRITTEN;
         }
+        return context.rewriteState;
     }
 
-    private static final RewriteState getState(Plan plan) {
-        Optional<RewriteState> state = plan.getMutableState(REWRITE_STATE_KEY);
-        return state.orElse(RewriteState.ENSURE_CHILDREN_REWRITTEN);
+    private static void setState(Plan plan, RewriteState state, int batchId) {
+        plan.setMutableState(REWRITE_STATE_KEY, new RewriteStateContext(state, batchId));
     }
 
-    private static final void setState(Plan plan, RewriteState state) {
-        plan.setMutableState(REWRITE_STATE_KEY, state);
+    private static class RewriteStateContext {
+        private final RewriteState rewriteState;
+        private final int batchId;
+
+        public RewriteStateContext(RewriteState rewriteState, int batchId) {
+            this.rewriteState = rewriteState;
+            this.batchId = batchId;
+        }
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java
index affbb9196cc..5e5acc29f66 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java
@@ -28,6 +28,8 @@ import org.apache.doris.nereids.pattern.Pattern;
 import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.trees.plans.Plan;
 
+import com.google.common.collect.ImmutableList;
+
 import java.util.List;
 
 /** PlanTreeRewriteJob */
@@ -43,7 +45,7 @@ public abstract class PlanTreeRewriteJob extends Job {
 
         boolean showPlanProcess = cascadesContext.showPlanProcess();
         for (Rule rule : rules) {
-            if (disableRules.contains(rule.getRuleType().type())) {
+            if (disableRules.get(rule.getRuleType().type())) {
                 continue;
             }
             Pattern<Plan> pattern = (Pattern<Plan>) rule.getPattern();
@@ -76,26 +78,50 @@ public abstract class PlanTreeRewriteJob extends Job {
         return new RewriteResult(false, plan);
     }
 
-    protected final Plan linkChildrenAndParent(Plan plan, RewriteJobContext rewriteJobContext) {
-        Plan newPlan = linkChildren(plan, rewriteJobContext.childrenContext);
-        rewriteJobContext.setResult(newPlan);
-        return newPlan;
-    }
-
-    protected final Plan linkChildren(Plan plan, RewriteJobContext[] childrenContext) {
-        boolean changed = false;
-        Plan[] newChildren = new Plan[childrenContext.length];
-        for (int i = 0; i < childrenContext.length; ++i) {
-            Plan result = childrenContext[i].result;
-            Plan oldChild = plan.child(i);
-            if (result != null && result != oldChild) {
-                newChildren[i] = result;
-                changed = true;
-            } else {
-                newChildren[i] = oldChild;
+    protected static Plan linkChildren(Plan plan, RewriteJobContext[] childrenContext) {
+        List<Plan> children = plan.children();
+        // loop unrolling
+        switch (children.size()) {
+            case 0: {
+                return plan;
+            }
+            case 1: {
+                RewriteJobContext child = childrenContext[0];
+                Plan firstResult = child == null ? plan.child(0) : child.result;
+                return firstResult == null || firstResult == children.get(0)
+                        ? plan : plan.withChildren(ImmutableList.of(firstResult));
+            }
+            case 2: {
+                RewriteJobContext left = childrenContext[0];
+                Plan firstResult = left == null ? plan.child(0) : left.result;
+                RewriteJobContext right = childrenContext[1];
+                Plan secondResult = right == null ? plan.child(1) : right.result;
+                Plan firstOrigin = children.get(0);
+                Plan secondOrigin = children.get(1);
+                boolean firstChanged = firstResult != null && firstResult != firstOrigin;
+                boolean secondChanged = secondResult != null && secondResult != secondOrigin;
+                if (firstChanged || secondChanged) {
+                    ImmutableList.Builder<Plan> newChildren = ImmutableList.builderWithExpectedSize(2);
+                    newChildren.add(firstChanged ? firstResult : firstOrigin);
+                    newChildren.add(secondChanged ? secondResult : secondOrigin);
+                    return plan.withChildren(newChildren.build());
+                } else {
+                    return plan;
+                }
+            }
+            default: {
+                boolean changed = false;
+                int i = 0;
+                Plan[] newChildren = new Plan[childrenContext.length];
+                for (Plan oldChild : children) {
+                    Plan result = childrenContext[i].result;
+                    changed = result != null && result != oldChild;
+                    newChildren[i] = changed ? result : oldChild;
+                    i++;
+                }
+                return changed ? plan.withChildren(newChildren) : plan;
             }
         }
-        return changed ? plan.withChildren(newChildren) : plan;
     }
 
     private String getCurrentPlanTreeString() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java
index d8dba41b378..14019bc885e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java
@@ -56,21 +56,44 @@ public class PlanTreeRewriteTopDownJob extends PlanTreeRewriteJob {
             RewriteJobContext newRewriteJobContext = rewriteJobContext.withChildrenVisited(true);
             pushJob(new PlanTreeRewriteTopDownJob(newRewriteJobContext, context, rules));
 
-            List<Plan> children = newRewriteJobContext.plan.children();
-            for (int i = children.size() - 1; i >= 0; i--) {
-                RewriteJobContext childRewriteJobContext = new RewriteJobContext(
-                        children.get(i), newRewriteJobContext, i, false);
-                // NOTICE: this relay on pull up cte anchor
-                if (!(rewriteJobContext.plan instanceof LogicalCTEAnchor)) {
-                    pushJob(new PlanTreeRewriteTopDownJob(childRewriteJobContext, context, rules));
-                }
+            // NOTICE: this relay on pull up cte anchor
+            if (!(this.rewriteJobContext.plan instanceof LogicalCTEAnchor)) {
+                pushChildrenJobs(newRewriteJobContext);
             }
         } else {
             // All the children part are already visited. Just link the children plan to the current node.
-            Plan result = linkChildrenAndParent(rewriteJobContext.plan, rewriteJobContext);
+            Plan result = linkChildren(rewriteJobContext.plan, rewriteJobContext.childrenContext);
+            rewriteJobContext.setResult(result);
             if (rewriteJobContext.parentContext == null) {
                 context.getCascadesContext().setRewritePlan(result);
             }
         }
     }
+
+    private void pushChildrenJobs(RewriteJobContext rewriteJobContext) {
+        List<Plan> children = rewriteJobContext.plan.children();
+        switch (children.size()) {
+            case 0: return;
+            case 1:
+                RewriteJobContext childRewriteJobContext = new RewriteJobContext(
+                        children.get(0), rewriteJobContext, 0, false, this.rewriteJobContext.batchId);
+                pushJob(new PlanTreeRewriteTopDownJob(childRewriteJobContext, context, rules));
+                return;
+            case 2:
+                RewriteJobContext rightRewriteJobContext = new RewriteJobContext(
+                        children.get(1), rewriteJobContext, 1, false, this.rewriteJobContext.batchId);
+                pushJob(new PlanTreeRewriteTopDownJob(rightRewriteJobContext, context, rules));
+
+                RewriteJobContext leftRewriteJobContext = new RewriteJobContext(
+                        children.get(0), rewriteJobContext, 0, false, this.rewriteJobContext.batchId);
+                pushJob(new PlanTreeRewriteTopDownJob(leftRewriteJobContext, context, rules));
+                return;
+            default:
+                for (int i = children.size() - 1; i >= 0; i--) {
+                    childRewriteJobContext = new RewriteJobContext(
+                            children.get(i), rewriteJobContext, i, false, this.rewriteJobContext.batchId);
+                    pushJob(new PlanTreeRewriteTopDownJob(childRewriteJobContext, context, rules));
+                }
+        }
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RewriteJobContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RewriteJobContext.java
index fb0475f7a61..060bb8edd62 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RewriteJobContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RewriteJobContext.java
@@ -25,6 +25,7 @@ import javax.annotation.Nullable;
 public class RewriteJobContext {
 
     final boolean childrenVisited;
+    final int batchId;
     final RewriteJobContext parentContext;
     final int childIndexInParentContext;
     final Plan plan;
@@ -33,7 +34,7 @@ public class RewriteJobContext {
 
     /** RewriteJobContext */
     public RewriteJobContext(Plan plan, @Nullable RewriteJobContext parentContext, int childIndexInParentContext,
-            boolean childrenVisited) {
+            boolean childrenVisited, int batchId) {
         this.plan = plan;
         this.parentContext = parentContext;
         this.childIndexInParentContext = childIndexInParentContext;
@@ -42,6 +43,7 @@ public class RewriteJobContext {
         if (parentContext != null) {
             parentContext.childrenContext[childIndexInParentContext] = this;
         }
+        this.batchId = batchId;
     }
 
     public void setResult(Plan result) {
@@ -49,15 +51,15 @@ public class RewriteJobContext {
     }
 
     public RewriteJobContext withChildrenVisited(boolean childrenVisited) {
-        return new RewriteJobContext(plan, parentContext, childIndexInParentContext, childrenVisited);
+        return new RewriteJobContext(plan, parentContext, childIndexInParentContext, childrenVisited, batchId);
     }
 
     public RewriteJobContext withPlan(Plan plan) {
-        return new RewriteJobContext(plan, parentContext, childIndexInParentContext, childrenVisited);
+        return new RewriteJobContext(plan, parentContext, childIndexInParentContext, childrenVisited, batchId);
     }
 
     public RewriteJobContext withPlanAndChildrenVisited(Plan plan, boolean childrenVisited) {
-        return new RewriteJobContext(plan, parentContext, childIndexInParentContext, childrenVisited);
+        return new RewriteJobContext(plan, parentContext, childIndexInParentContext, childrenVisited, batchId);
     }
 
     public boolean isRewriteRoot() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java
index 6bc055a68aa..d352dfee4a0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java
@@ -27,9 +27,11 @@ import org.apache.doris.nereids.trees.plans.Plan;
 
 import java.util.List;
 import java.util.Objects;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /** RootPlanTreeRewriteJob */
 public class RootPlanTreeRewriteJob implements RewriteJob {
+    private static final AtomicInteger BATCH_ID = new AtomicInteger();
 
     private final List<Rule> rules;
     private final RewriteJobBuilder rewriteJobBuilder;
@@ -47,7 +49,9 @@ public class RootPlanTreeRewriteJob implements RewriteJob {
         // get plan from the cascades context
         Plan root = cascadesContext.getRewritePlan();
         // write rewritten root plan to cascades context by the RootRewriteJobContext
-        RootRewriteJobContext rewriteJobContext = new RootRewriteJobContext(root, false, context);
+        int batchId = BATCH_ID.incrementAndGet();
+        RootRewriteJobContext rewriteJobContext = new RootRewriteJobContext(
+                root, false, context, batchId);
         Job rewriteJob = rewriteJobBuilder.build(rewriteJobContext, context, rules);
 
         context.getScheduleContext().pushJob(rewriteJob);
@@ -71,8 +75,8 @@ public class RootPlanTreeRewriteJob implements RewriteJob {
 
         private final JobContext jobContext;
 
-        RootRewriteJobContext(Plan plan, boolean childrenVisited, JobContext jobContext) {
-            super(plan, null, -1, childrenVisited);
+        RootRewriteJobContext(Plan plan, boolean childrenVisited, JobContext jobContext, int batchId) {
+            super(plan, null, -1, childrenVisited, batchId);
             this.jobContext = Objects.requireNonNull(jobContext, "jobContext cannot be null");
             jobContext.getCascadesContext().setCurrentRootRewriteJobContext(this);
         }
@@ -89,17 +93,17 @@ public class RootPlanTreeRewriteJob implements RewriteJob {
 
         @Override
         public RewriteJobContext withChildrenVisited(boolean childrenVisited) {
-            return new RootRewriteJobContext(plan, childrenVisited, jobContext);
+            return new RootRewriteJobContext(plan, childrenVisited, jobContext, batchId);
         }
 
         @Override
         public RewriteJobContext withPlan(Plan plan) {
-            return new RootRewriteJobContext(plan, childrenVisited, jobContext);
+            return new RootRewriteJobContext(plan, childrenVisited, jobContext, batchId);
         }
 
         @Override
         public RewriteJobContext withPlanAndChildrenVisited(Plan plan, boolean childrenVisited) {
-            return new RootRewriteJobContext(plan, childrenVisited, jobContext);
+            return new RootRewriteJobContext(plan, childrenVisited, jobContext, batchId);
         }
 
         /** linkChildren */
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ExpressionPatternRules.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ExpressionPatternRules.java
new file mode 100644
index 00000000000..523540e6435
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ExpressionPatternRules.java
@@ -0,0 +1,112 @@
+// 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.pattern;
+
+import org.apache.doris.nereids.rules.expression.ExpressionMatchingContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatchRule;
+import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.trees.expressions.Expression;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.lang.reflect.Field;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+
+/** ExpressionPatternMapping */
+public class ExpressionPatternRules extends TypeMappings<Expression, ExpressionPatternMatchRule> {
+    private static final Logger LOG = LogManager.getLogger(ExpressionPatternRules.class);
+
+    public ExpressionPatternRules(List<ExpressionPatternMatchRule> typeMappings) {
+        super(typeMappings);
+    }
+
+    @Override
+    protected Set<Class<? extends Expression>> getChildrenClasses(Class<? extends Expression> clazz) {
+        return org.apache.doris.nereids.pattern.GeneratedExpressionRelations.CHILDREN_CLASS_MAP.get(clazz);
+    }
+
+    /** matchesAndApply */
+    public Optional<Expression> matchesAndApply(Expression expr, ExpressionRewriteContext context, Expression parent) {
+        List<ExpressionPatternMatchRule> rules = singleMappings.get(expr.getClass());
+        ExpressionMatchingContext<Expression> matchingContext
+                = new ExpressionMatchingContext<>(expr, parent, context);
+        switch (rules.size()) {
+            case 0: {
+                for (ExpressionPatternMatchRule multiMatchRule : multiMappings) {
+                    if (multiMatchRule.matchesTypeAndPredicates(matchingContext)) {
+                        Expression newExpr = multiMatchRule.apply(matchingContext);
+                        if (!newExpr.equals(expr)) {
+                            if (context.cascadesContext.isEnableExprTrace()) {
+                                traceExprChanged(multiMatchRule, expr, newExpr);
+                            }
+                            return Optional.of(newExpr);
+                        }
+                    }
+                }
+                return Optional.empty();
+            }
+            case 1: {
+                ExpressionPatternMatchRule rule = rules.get(0);
+                if (rule.matchesPredicates(matchingContext)) {
+                    Expression newExpr = rule.apply(matchingContext);
+                    if (!newExpr.equals(expr)) {
+                        if (context.cascadesContext.isEnableExprTrace()) {
+                            traceExprChanged(rule, expr, newExpr);
+                        }
+                        return Optional.of(newExpr);
+                    }
+                }
+                return Optional.empty();
+            }
+            default: {
+                for (ExpressionPatternMatchRule rule : rules) {
+                    if (rule.matchesPredicates(matchingContext)) {
+                        Expression newExpr = rule.apply(matchingContext);
+                        if (!expr.equals(newExpr)) {
+                            if (context.cascadesContext.isEnableExprTrace()) {
+                                traceExprChanged(rule, expr, newExpr);
+                            }
+                            return Optional.of(newExpr);
+                        }
+                    }
+                }
+                return Optional.empty();
+            }
+        }
+    }
+
+    private static void traceExprChanged(ExpressionPatternMatchRule rule, Expression expr, Expression newExpr) {
+        try {
+            Field[] declaredFields = (rule.matchingAction).getClass().getDeclaredFields();
+            Class<?> ruleClass;
+            if (declaredFields.length == 0) {
+                ruleClass = rule.matchingAction.getClass();
+            } else {
+                Field field = declaredFields[0];
+                field.setAccessible(true);
+                ruleClass = field.get(rule.matchingAction).getClass();
+            }
+            LOG.info("RULE: " + ruleClass + "\nbefore: " + expr + "\nafter: " + newExpr);
+        } catch (Throwable t) {
+            LOG.error(t.getMessage(), t);
+        }
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ExpressionPatternTraverseListeners.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ExpressionPatternTraverseListeners.java
new file mode 100644
index 00000000000..3f3640a43bf
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ExpressionPatternTraverseListeners.java
@@ -0,0 +1,112 @@
+// 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.pattern;
+
+import org.apache.doris.nereids.rules.expression.ExpressionMatchingContext;
+import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionTraverseListener;
+import org.apache.doris.nereids.rules.expression.ExpressionTraverseListenerMapping;
+import org.apache.doris.nereids.trees.expressions.Expression;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+/** ExpressionPatternTraverseListeners */
+public class ExpressionPatternTraverseListeners
+        extends TypeMappings<Expression, ExpressionTraverseListenerMapping> {
+    public ExpressionPatternTraverseListeners(
+            List<ExpressionTraverseListenerMapping> typeMappings) {
+        super(typeMappings);
+    }
+
+    @Override
+    protected Set<Class<? extends Expression>> getChildrenClasses(Class<? extends Expression> clazz) {
+        return org.apache.doris.nereids.pattern.GeneratedExpressionRelations.CHILDREN_CLASS_MAP.get(clazz);
+    }
+
+    /** matchesAndCombineListener */
+    public @Nullable CombinedListener matchesAndCombineListeners(
+            Expression expr, ExpressionRewriteContext context, Expression parent) {
+        List<ExpressionTraverseListenerMapping> listenerSingleMappings = singleMappings.get(expr.getClass());
+        ExpressionMatchingContext<Expression> matchingContext
+                = new ExpressionMatchingContext<>(expr, parent, context);
+        switch (listenerSingleMappings.size()) {
+            case 0: {
+                ImmutableList.Builder<ExpressionTraverseListener<Expression>> matchedListeners
+                        = ImmutableList.builder();
+                for (ExpressionTraverseListenerMapping multiMapping : multiMappings) {
+                    if (multiMapping.matchesTypeAndPredicates(matchingContext)) {
+                        matchedListeners.add(multiMapping.listener);
+                    }
+                }
+                return CombinedListener.tryCombine(matchedListeners.build(), matchingContext);
+            }
+            case 1: {
+                ExpressionTraverseListenerMapping listenerMapping = listenerSingleMappings.get(0);
+                if (listenerMapping.matchesPredicates(matchingContext)) {
+                    return CombinedListener.tryCombine(ImmutableList.of(listenerMapping.listener), matchingContext);
+                }
+                return null;
+            }
+            default: {
+                ImmutableList.Builder<ExpressionTraverseListener<Expression>> matchedListeners
+                        = ImmutableList.builder();
+                for (ExpressionTraverseListenerMapping singleMapping : listenerSingleMappings) {
+                    if (singleMapping.matchesPredicates(matchingContext)) {
+                        matchedListeners.add(singleMapping.listener);
+                    }
+                }
+                return CombinedListener.tryCombine(matchedListeners.build(), matchingContext);
+            }
+        }
+    }
+
+    /** CombinedListener */
+    public static class CombinedListener {
+        private final ExpressionMatchingContext<Expression> context;
+        private final List<ExpressionTraverseListener<Expression>> listeners;
+
+        /** CombinedListener */
+        public CombinedListener(ExpressionMatchingContext<Expression> context,
+                List<ExpressionTraverseListener<Expression>> listeners) {
+            this.context = context;
+            this.listeners = listeners;
+        }
+
+        public static @Nullable CombinedListener tryCombine(
+                List<ExpressionTraverseListener<Expression>> listenerMappings,
+                ExpressionMatchingContext<Expression> context) {
+            return listenerMappings.isEmpty() ? null : new CombinedListener(context, listenerMappings);
+        }
+
+        public void onEnter() {
+            for (ExpressionTraverseListener<Expression> listener : listeners) {
+                listener.onEnter(context);
+            }
+        }
+
+        public void onExit(Expression rewritten) {
+            for (ExpressionTraverseListener<Expression> listener : listeners) {
+                listener.onExit(context, rewritten);
+            }
+        }
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ParentTypeIdMapping.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ParentTypeIdMapping.java
new file mode 100644
index 00000000000..b4623e10523
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/ParentTypeIdMapping.java
@@ -0,0 +1,59 @@
+// 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.pattern;
+
+import org.apache.doris.nereids.trees.expressions.LessThanEqual;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/** ParentTypeIdMapping */
+public class ParentTypeIdMapping {
+
+    private final AtomicInteger idGenerator = new AtomicInteger();
+    private final Map<Class<?>, Integer> classId = new ConcurrentHashMap<>(8192);
+
+    /** getId */
+    public int getId(Class<?> clazz) {
+        Integer id = classId.get(clazz);
+        if (id != null) {
+            return id;
+        }
+        return ensureClassHasId(clazz);
+    }
+
+    private int ensureClassHasId(Class<?> clazz) {
+        Class<?> superClass = clazz.getSuperclass();
+        if (superClass != null) {
+            ensureClassHasId(superClass);
+        }
+
+        for (Class<?> interfaceClass : clazz.getInterfaces()) {
+            ensureClassHasId(interfaceClass);
+        }
+
+        return classId.computeIfAbsent(clazz, c -> idGenerator.incrementAndGet());
+    }
+
+    public static void main(String[] args) {
+        ParentTypeIdMapping mapping = new ParentTypeIdMapping();
+        int id = mapping.getId(LessThanEqual.class);
+        System.out.println(id);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/Pattern.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/Pattern.java
index c47dcd6a725..91dd87ba457 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/Pattern.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/Pattern.java
@@ -152,6 +152,10 @@ public class Pattern<TYPE extends Plan>
         if (this instanceof SubTreePattern) {
             return matchPredicates((TYPE) plan);
         }
+        return matchChildrenAndSelfPredicates(plan, childPatternNum);
+    }
+
+    private boolean matchChildrenAndSelfPredicates(Plan plan, int childPatternNum) {
         List<Plan> childrenPlan = plan.children();
         for (int i = 0; i < childrenPlan.size(); i++) {
             Plan child = childrenPlan.get(i);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/TypeMappings.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/TypeMappings.java
new file mode 100644
index 00000000000..4eb5ffc76d2
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/TypeMappings.java
@@ -0,0 +1,133 @@
+// 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.pattern;
+
+import org.apache.doris.nereids.pattern.TypeMappings.TypeMapping;
+import org.apache.doris.nereids.util.Utils;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+
+import java.lang.reflect.Modifier;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+/** ExpressionPatternMappings */
+public abstract class TypeMappings<K, T extends TypeMapping<K>> {
+    protected final ListMultimap<Class<? extends K>, T> singleMappings;
+    protected final List<T> multiMappings;
+
+    /** ExpressionPatternMappings */
+    public TypeMappings(List<T> typeMappings) {
+        this.singleMappings = ArrayListMultimap.create();
+        this.multiMappings = Lists.newArrayList();
+
+        for (T mapping : typeMappings) {
+            Set<Class<? extends K>> childrenClasses = getChildrenClasses(mapping.getType());
+            if (childrenClasses == null || childrenClasses.isEmpty()) {
+                // add some expressions which no child class
+                // e.g. LessThanEqual
+                addSimpleMapping(mapping);
+            } else if (childrenClasses.size() <= 100) {
+                // add some expressions which have children classes
+                // e.g. ComparisonPredicate will be expanded to
+                //      ruleMappings.put(LessThanEqual.class, rule);
+                //      ruleMappings.put(LessThan.class, rule);
+                //      ruleMappings.put(GreaterThan.class, rule);
+                //      ruleMappings.put(GreaterThanEquals.class, rule);
+                //      ...
+                addThisAndChildrenMapping(mapping, childrenClasses);
+            } else {
+                // some expressions have lots of children classes, e.g. Expression, ExpressionTrait, BinaryExpression,
+                // we will not expand this types to child class, but also add this rules to other type matching.
+                // for example, if we have three rules to matches this types: LessThanEqual, Abs and Expression,
+                // then the ruleMappings would be:
+                //   {
+                //      LessThanEqual.class: [rule_of_LessThanEqual, rule_of_Expression],
+                //      Abs.class: [rule_of_Abs, rule_of_Expression]
+                //   }
+                //
+                // and the multiMatchRules would be: [rule_of_Expression]
+                //
+                // if we matches `a <= 1`, there have two rules would be applied because
+                // ruleMappings.get(LessThanEqual.class) return two rules;
+                // if we matches `a = 1`, ruleMappings.get(EqualTo.class) will return empty rules, so we use
+                // all the rules in multiMatchRules to matches and apply, the rule_of_Expression will be applied.
+                addMultiMapping(mapping);
+            }
+        }
+    }
+
+    public @Nullable List<T> get(Class<? extends K> clazz) {
+        return singleMappings.get(clazz);
+    }
+
+    private void addSimpleMapping(T typeMapping) {
+        Class<? extends K> clazz = typeMapping.getType();
+        int modifiers = clazz.getModifiers();
+        if (!Modifier.isAbstract(modifiers)) {
+            addSingleMapping(clazz, typeMapping);
+        }
+    }
+
+    private void addThisAndChildrenMapping(
+            T typeMapping, Set<Class<? extends K>> childrenClasses) {
+        Class<? extends K> clazz = typeMapping.getType();
+        if (!Modifier.isAbstract(clazz.getModifiers())) {
+            addSingleMapping(clazz, typeMapping);
+        }
+
+        for (Class<? extends K> childrenClass : childrenClasses) {
+            if (!Modifier.isAbstract(childrenClass.getModifiers())) {
+                addSingleMapping(childrenClass, typeMapping);
+            }
+        }
+    }
+
+    private void addMultiMapping(T multiMapping) {
+        multiMappings.add(multiMapping);
+
+        Set<Class<? extends K>> existSingleMappingTypes = Utils.fastToImmutableSet(singleMappings.keySet());
+        for (Class<? extends K> existSingleType : existSingleMappingTypes) {
+            Class<? extends K> type = multiMapping.getType();
+            if (type.isAssignableFrom(existSingleType)) {
+                singleMappings.put(existSingleType, multiMapping);
+            }
+        }
+    }
+
+    private void addSingleMapping(Class<? extends K> clazz, T singleMapping) {
+        if (!singleMappings.containsKey(clazz) && !multiMappings.isEmpty()) {
+            for (T multiMapping : multiMappings) {
+                if (multiMapping.getType().isAssignableFrom(clazz)) {
+                    singleMappings.put(clazz, multiMapping);
+                }
+            }
+        }
+        singleMappings.put(clazz, singleMapping);
+    }
+
+    protected abstract Set<Class<? extends K>> getChildrenClasses(Class<? extends K> clazz);
+
+    /** TypeMapping */
+    public interface TypeMapping<K> {
+        Class<? extends K> getType();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/ExpressionTypeMappingGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/ExpressionTypeMappingGenerator.java
new file mode 100644
index 00000000000..c5a923153df
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/ExpressionTypeMappingGenerator.java
@@ -0,0 +1,159 @@
+// 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.pattern.generator;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.stream.Collectors;
+import javax.annotation.processing.ProcessingEnvironment;
+import javax.tools.StandardLocation;
+
+/** ExpressionTypeMappingGenerator */
+public class ExpressionTypeMappingGenerator {
+    private final JavaAstAnalyzer analyzer;
+
+    public ExpressionTypeMappingGenerator(JavaAstAnalyzer javaAstAnalyzer) {
+        this.analyzer = javaAstAnalyzer;
+    }
+
+    public JavaAstAnalyzer getAnalyzer() {
+        return analyzer;
+    }
+
+    /** generate */
+    public void generate(ProcessingEnvironment processingEnv) throws IOException {
+        Set<String> superExpressions = findSuperExpression();
+        Map<String, Set<String>> childrenNameMap = analyzer.getChildrenNameMap();
+        Map<String, Set<String>> parentNameMap = analyzer.getParentNameMap();
+        String code = generateCode(childrenNameMap, parentNameMap, superExpressions);
+        generateFile(processingEnv, code);
+    }
+
+    private void generateFile(ProcessingEnvironment processingEnv, String code) throws IOException {
+        File generatePatternFile = new File(processingEnv.getFiler()
+                .getResource(StandardLocation.SOURCE_OUTPUT, "org.apache.doris.nereids.pattern",
+                        "GeneratedExpressionRelations.java").toUri());
+        if (generatePatternFile.exists()) {
+            generatePatternFile.delete();
+        }
+        if (!generatePatternFile.getParentFile().exists()) {
+            generatePatternFile.getParentFile().mkdirs();
+        }
+
+        // bypass create file for processingEnv.getFiler(), compile GeneratePatterns in next compile term
+        try (BufferedWriter bufferedWriter = new BufferedWriter(new FileWriter(generatePatternFile))) {
+            bufferedWriter.write(code);
+        }
+    }
+
+    private Set<String> findSuperExpression() {
+        Map<String, Set<String>> parentNameMap = analyzer.getParentNameMap();
+        Map<String, Set<String>> childrenNameMap = analyzer.getChildrenNameMap();
+        Set<String> superExpressions = Sets.newLinkedHashSet();
+        for (Entry<String, Set<String>> entry : childrenNameMap.entrySet()) {
+            String parentName = entry.getKey();
+            Set<String> childrenNames = entry.getValue();
+
+            if (parentName.startsWith("org.apache.doris.nereids.trees.expressions.")) {
+                for (String childrenName : childrenNames) {
+                    Set<String> parentNames = parentNameMap.get(childrenName);
+                    if (parentNames != null
+                            && parentNames.contains("org.apache.doris.nereids.trees.expressions.Expression")) {
+                        superExpressions.add(parentName);
+                        break;
+                    }
+                }
+            }
+        }
+        return superExpressions;
+    }
+
+    private String generateCode(Map<String, Set<String>> childrenNameMap,
+            Map<String, Set<String>> parentNameMap, Set<String> superExpressions) {
+        String generateCode
+                = "// Licensed to the Apache Software Foundation (ASF) under one\n"
+                + "// or more contributor license agreements.  See the NOTICE file\n"
+                + "// distributed with this work for additional information\n"
+                + "// regarding copyright ownership.  The ASF licenses this file\n"
+                + "// to you under the Apache License, Version 2.0 (the\n"
+                + "// \"License\"); you may not use this file except in compliance\n"
+                + "// with the License.  You may obtain a copy of the License at\n"
+                + "//\n"
+                + "//   http://www.apache.org/licenses/LICENSE-2.0\n"
+                + "//\n"
+                + "// Unless required by applicable law or agreed to in writing,\n"
+                + "// software distributed under the License is distributed on an\n"
+                + "// \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n"
+                + "// KIND, either express or implied.  See the License for the\n"
+                + "// specific language governing permissions and limitations\n"
+                + "// under the License.\n"
+                + "\n"
+                + "package org.apache.doris.nereids.pattern;\n"
+                + "\n"
+                + "import org.apache.doris.nereids.trees.expressions.Expression;\n"
+                + "\n"
+                + "import com.google.common.collect.ImmutableMap;\n"
+                + "import com.google.common.collect.ImmutableSet;\n"
+                + "\n"
+                + "import java.util.Map;\n"
+                + "import java.util.Set;\n"
+                + "\n";
+        generateCode += "/** GeneratedExpressionRelations */\npublic class GeneratedExpressionRelations {\n";
+        String childrenClassesGenericType = "<Class<?>, Set<Class<? extends Expression>>>";
+        generateCode +=
+                "    public static final Map" + childrenClassesGenericType + " CHILDREN_CLASS_MAP;\n\n";
+        generateCode +=
+                "    static {\n"
+              + "        ImmutableMap.Builder" + childrenClassesGenericType + " childrenClassesBuilder\n"
+              + "                = ImmutableMap.builderWithExpectedSize(" + childrenNameMap.size() + ");\n";
+
+        for (String superExpression : superExpressions) {
+            Set<String> childrenClasseSet = childrenNameMap.get(superExpression)
+                    .stream()
+                    .filter(childClass -> parentNameMap.get(childClass)
+                            .contains("org.apache.doris.nereids.trees.expressions.Expression")
+                    )
+                    .collect(Collectors.toSet());
+
+            List<String> childrenClasses = Lists.newArrayList(childrenClasseSet);
+            Collections.sort(childrenClasses, Comparator.naturalOrder());
+
+            String childClassesString = childrenClasses.stream()
+                    .map(childClass -> "                    " + childClass + ".class")
+                    .collect(Collectors.joining(",\n"));
+            generateCode += "        childrenClassesBuilder.put(\n                " + superExpression
+                    + ".class,\n                ImmutableSet.<Class<? extends Expression>>of(\n" + childClassesString
+                    + "\n                )\n        );\n\n";
+        }
+
+        generateCode += "        CHILDREN_CLASS_MAP = childrenClassesBuilder.build();\n";
+
+        return generateCode + "    }\n}\n";
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PatternGeneratorAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/JavaAstAnalyzer.java
similarity index 75%
rename from fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PatternGeneratorAnalyzer.java
rename to fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/JavaAstAnalyzer.java
index f4a9d128087..cce69151ca2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PatternGeneratorAnalyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/JavaAstAnalyzer.java
@@ -29,25 +29,24 @@ import org.apache.doris.nereids.pattern.generator.javaast.TypeType;
 
 import com.google.common.base.Joiner;
 
-import java.lang.reflect.Modifier;
 import java.util.ArrayList;
 import java.util.IdentityHashMap;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Optional;
 import java.util.Set;
-import java.util.stream.Collectors;
 
-/**
- * used to analyze plan class extends hierarchy and then generated pattern builder methods.
- */
-public class PatternGeneratorAnalyzer {
-    private final Map<String, TypeDeclaration> name2Ast = new LinkedHashMap<>();
-    private final IdentityHashMap<TypeDeclaration, String> ast2Name = new IdentityHashMap<>();
-    private final IdentityHashMap<TypeDeclaration, Map<String, String>> ast2Import = new IdentityHashMap<>();
-    private final IdentityHashMap<TypeDeclaration, Set<String>> parentClassMap = new IdentityHashMap<>();
+/** JavaAstAnalyzer */
+public class JavaAstAnalyzer {
+    protected final Map<String, TypeDeclaration> name2Ast = new LinkedHashMap<>();
+    protected final IdentityHashMap<TypeDeclaration, String> ast2Name = new IdentityHashMap<>();
+    protected final IdentityHashMap<TypeDeclaration, Map<String, String>> ast2Import = new IdentityHashMap<>();
+    protected final IdentityHashMap<TypeDeclaration, Set<String>> parentClassMap = new IdentityHashMap<>();
+    protected final Map<String, Set<String>> parentNameMap = new LinkedHashMap<>();
+    protected final Map<String, Set<String>> childrenNameMap = new LinkedHashMap<>();
 
     /** add java AST. */
     public void addAsts(List<TypeDeclaration> typeDeclarations) {
@@ -56,14 +55,20 @@ public class PatternGeneratorAnalyzer {
         }
     }
 
-    /** generate pattern methods. */
-    public String generatePatterns(String className, String parentClassName, boolean isMemoPattern) {
-        analyzeImport();
-        analyzeParentClass();
-        return doGenerate(className, parentClassName, isMemoPattern);
+    public IdentityHashMap<TypeDeclaration, Set<String>> getParentClassMap() {
+        return parentClassMap;
+    }
+
+    public Map<String, Set<String>> getParentNameMap() {
+        return parentNameMap;
     }
 
-    Optional<TypeDeclaration> getType(TypeDeclaration typeDeclaration, TypeType type) {
+    public Map<String, Set<String>> getChildrenNameMap() {
+        return childrenNameMap;
+    }
+
+    /** getType */
+    public Optional<TypeDeclaration> getType(TypeDeclaration typeDeclaration, TypeType type) {
         String typeName = analyzeClass(new LinkedHashSet<>(), typeDeclaration, type);
         if (typeName != null) {
             TypeDeclaration ast = name2Ast.get(typeName);
@@ -73,34 +78,11 @@ public class PatternGeneratorAnalyzer {
         return Optional.empty();
     }
 
-    private String doGenerate(String className, String parentClassName, boolean isMemoPattern) {
-        Map<ClassDeclaration, Set<String>> planClassMap = parentClassMap.entrySet().stream()
-                .filter(kv -> kv.getValue().contains("org.apache.doris.nereids.trees.plans.Plan"))
-                .filter(kv -> !kv.getKey().name.equals("GroupPlan"))
-                .filter(kv -> !Modifier.isAbstract(kv.getKey().modifiers.mod)
-                        && kv.getKey() instanceof ClassDeclaration)
-                .collect(Collectors.toMap(kv -> (ClassDeclaration) kv.getKey(), kv -> kv.getValue()));
-
-        List<PatternGenerator> generators = planClassMap.entrySet()
-                .stream()
-                .map(kv -> PatternGenerator.create(this, kv.getKey(), kv.getValue(), isMemoPattern))
-                .filter(Optional::isPresent)
-                .map(Optional::get)
-                .sorted((g1, g2) -> {
-                    // logical first
-                    if (g1.isLogical() != g2.isLogical()) {
-                        return g1.isLogical() ? -1 : 1;
-                    }
-                    // leaf first
-                    if (g1.childrenNum() != g2.childrenNum()) {
-                        return g1.childrenNum() - g2.childrenNum();
-                    }
-                    // string dict sort
-                    return g1.opType.name.compareTo(g2.opType.name);
-                })
-                .collect(Collectors.toList());
-
-        return PatternGenerator.generateCode(className, parentClassName, generators, this, isMemoPattern);
+    protected void analyze() {
+        analyzeImport();
+        analyzeParentClass();
+        analyzeParentName();
+        analyzeChildrenName();
     }
 
     private void analyzeImport() {
@@ -148,7 +130,28 @@ public class PatternGeneratorAnalyzer {
         parentClasses.addAll(currentParentClasses);
     }
 
-    String analyzeClass(Set<String> parentClasses, TypeDeclaration typeDeclaration, TypeType type) {
+    private void analyzeParentName() {
+        for (Entry<TypeDeclaration, Set<String>> entry : parentClassMap.entrySet()) {
+            String parentName = entry.getKey().getFullQualifiedName();
+            parentNameMap.put(parentName, entry.getValue());
+        }
+    }
+
+    private void analyzeChildrenName() {
+        for (Entry<String, TypeDeclaration> entry : name2Ast.entrySet()) {
+            Set<String> parentNames = parentClassMap.get(entry.getValue());
+            for (String parentName : parentNames) {
+                Set<String> childrenNames = childrenNameMap.get(parentName);
+                if (childrenNames == null) {
+                    childrenNames = new LinkedHashSet<>();
+                    childrenNameMap.put(parentName, childrenNames);
+                }
+                childrenNames.add(entry.getKey());
+            }
+        }
+    }
+
+    private String analyzeClass(Set<String> parentClasses, TypeDeclaration typeDeclaration, TypeType type) {
         if (type.classOrInterfaceType.isPresent()) {
             List<String> identifiers = new ArrayList<>();
             ClassOrInterfaceType classOrInterfaceType = type.classOrInterfaceType.get();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalBinaryPatternGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalBinaryPatternGenerator.java
index bec3efa270a..8e05a87ad7d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalBinaryPatternGenerator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalBinaryPatternGenerator.java
@@ -23,9 +23,9 @@ import java.util.Set;
 import java.util.TreeSet;
 
 /** used to generate pattern for LogicalBinary. */
-public class LogicalBinaryPatternGenerator extends PatternGenerator {
+public class LogicalBinaryPatternGenerator extends PlanPatternGenerator {
 
-    public LogicalBinaryPatternGenerator(PatternGeneratorAnalyzer analyzer,
+    public LogicalBinaryPatternGenerator(PlanPatternGeneratorAnalyzer analyzer,
             ClassDeclaration opType, Set<String> parentClass, boolean isMemoPattern) {
         super(analyzer, opType, parentClass, isMemoPattern);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalLeafPatternGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalLeafPatternGenerator.java
index fd7b30a8e6f..b82ac81d420 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalLeafPatternGenerator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalLeafPatternGenerator.java
@@ -23,9 +23,9 @@ import java.util.Set;
 import java.util.TreeSet;
 
 /** used to generate pattern for LogicalLeaf. */
-public class LogicalLeafPatternGenerator extends PatternGenerator {
+public class LogicalLeafPatternGenerator extends PlanPatternGenerator {
 
-    public LogicalLeafPatternGenerator(PatternGeneratorAnalyzer analyzer,
+    public LogicalLeafPatternGenerator(PlanPatternGeneratorAnalyzer analyzer,
             ClassDeclaration opType, Set<String> parentClass, boolean isMemoPattern) {
         super(analyzer, opType, parentClass, isMemoPattern);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalUnaryPatternGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalUnaryPatternGenerator.java
index 8ecb7c14e10..d2f2b61bf96 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalUnaryPatternGenerator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/LogicalUnaryPatternGenerator.java
@@ -23,9 +23,9 @@ import java.util.Set;
 import java.util.TreeSet;
 
 /** used to generate pattern for LogicalUnary. */
-public class LogicalUnaryPatternGenerator extends PatternGenerator {
+public class LogicalUnaryPatternGenerator extends PlanPatternGenerator {
 
-    public LogicalUnaryPatternGenerator(PatternGeneratorAnalyzer analyzer,
+    public LogicalUnaryPatternGenerator(PlanPatternGeneratorAnalyzer analyzer,
             ClassDeclaration opType, Set<String> parentClass, boolean isMemoPattern) {
         super(analyzer, opType, parentClass, isMemoPattern);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PatternDescribableProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PatternDescribableProcessor.java
index 42cf82e3c01..5ba81bbb96b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PatternDescribableProcessor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PatternDescribableProcessor.java
@@ -60,12 +60,12 @@ import javax.tools.StandardLocation;
 @SupportedSourceVersion(SourceVersion.RELEASE_8)
 @SupportedAnnotationTypes("org.apache.doris.nereids.pattern.generator.PatternDescribable")
 public class PatternDescribableProcessor extends AbstractProcessor {
-    private List<File> planPaths;
+    private List<File> paths;
 
     @Override
     public synchronized void init(ProcessingEnvironment processingEnv) {
         super.init(processingEnv);
-        this.planPaths = Arrays.stream(processingEnv.getOptions().get("planPath").split(","))
+        this.paths = Arrays.stream(processingEnv.getOptions().get("path").split(","))
                 .map(path -> path.trim())
                 .filter(path -> !path.isEmpty())
                 .collect(Collectors.toSet())
@@ -80,15 +80,25 @@ public class PatternDescribableProcessor extends AbstractProcessor {
             return false;
         }
         try {
-            List<File> planFiles = findJavaFiles(planPaths);
-            PatternGeneratorAnalyzer patternGeneratorAnalyzer = new PatternGeneratorAnalyzer();
-            for (File file : planFiles) {
+            List<File> javaFiles = findJavaFiles(paths);
+            JavaAstAnalyzer javaAstAnalyzer = new JavaAstAnalyzer();
+            for (File file : javaFiles) {
                 List<TypeDeclaration> asts = parseJavaFile(file);
-                patternGeneratorAnalyzer.addAsts(asts);
+                javaAstAnalyzer.addAsts(asts);
             }
 
-            doGenerate("GeneratedMemoPatterns", "MemoPatterns", true, patternGeneratorAnalyzer);
-            doGenerate("GeneratedPlanPatterns", "PlanPatterns", false, patternGeneratorAnalyzer);
+            javaAstAnalyzer.analyze();
+
+            ExpressionTypeMappingGenerator expressionTypeMappingGenerator
+                    = new ExpressionTypeMappingGenerator(javaAstAnalyzer);
+            expressionTypeMappingGenerator.generate(processingEnv);
+
+            PlanTypeMappingGenerator planTypeMappingGenerator = new PlanTypeMappingGenerator(javaAstAnalyzer);
+            planTypeMappingGenerator.generate(processingEnv);
+
+            PlanPatternGeneratorAnalyzer patternGeneratorAnalyzer = new PlanPatternGeneratorAnalyzer(javaAstAnalyzer);
+            generatePlanPatterns("GeneratedMemoPatterns", "MemoPatterns", true, patternGeneratorAnalyzer);
+            generatePlanPatterns("GeneratedPlanPatterns", "PlanPatterns", false, patternGeneratorAnalyzer);
         } catch (Throwable t) {
             String exceptionMsg = Throwables.getStackTraceAsString(t);
             processingEnv.getMessager().printMessage(Kind.ERROR,
@@ -97,8 +107,12 @@ public class PatternDescribableProcessor extends AbstractProcessor {
         return false;
     }
 
-    private void doGenerate(String className, String parentClassName, boolean isMemoPattern,
-            PatternGeneratorAnalyzer patternGeneratorAnalyzer) throws IOException {
+    private void generateExpressionTypeMapping() {
+
+    }
+
+    private void generatePlanPatterns(String className, String parentClassName, boolean isMemoPattern,
+            PlanPatternGeneratorAnalyzer patternGeneratorAnalyzer) throws IOException {
         String generatePatternCode = patternGeneratorAnalyzer.generatePatterns(
                 className, parentClassName, isMemoPattern);
         File generatePatternFile = new File(processingEnv.getFiler()
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalBinaryPatternGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalBinaryPatternGenerator.java
index 72a31557495..08e639a924d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalBinaryPatternGenerator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalBinaryPatternGenerator.java
@@ -23,9 +23,9 @@ import java.util.Set;
 import java.util.TreeSet;
 
 /** used to generate pattern for PhysicalBinary. */
-public class PhysicalBinaryPatternGenerator extends PatternGenerator {
+public class PhysicalBinaryPatternGenerator extends PlanPatternGenerator {
 
-    public PhysicalBinaryPatternGenerator(PatternGeneratorAnalyzer analyzer,
+    public PhysicalBinaryPatternGenerator(PlanPatternGeneratorAnalyzer analyzer,
             ClassDeclaration opType, Set<String> parentClass, boolean isMemoPattern) {
         super(analyzer, opType, parentClass, isMemoPattern);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalLeafPatternGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalLeafPatternGenerator.java
index f75746b5142..27a94edacad 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalLeafPatternGenerator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalLeafPatternGenerator.java
@@ -23,9 +23,9 @@ import java.util.Set;
 import java.util.TreeSet;
 
 /** used to generate pattern for PhysicalLeaf. */
-public class PhysicalLeafPatternGenerator extends PatternGenerator {
+public class PhysicalLeafPatternGenerator extends PlanPatternGenerator {
 
-    public PhysicalLeafPatternGenerator(PatternGeneratorAnalyzer analyzer,
+    public PhysicalLeafPatternGenerator(PlanPatternGeneratorAnalyzer analyzer,
             ClassDeclaration opType, Set<String> parentClass, boolean isMemoPattern) {
         super(analyzer, opType, parentClass, isMemoPattern);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalUnaryPatternGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalUnaryPatternGenerator.java
index 4254e28ee43..f69de7e9d6a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalUnaryPatternGenerator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PhysicalUnaryPatternGenerator.java
@@ -23,9 +23,9 @@ import java.util.Set;
 import java.util.TreeSet;
 
 /** used to generate pattern for PhysicalUnary. */
-public class PhysicalUnaryPatternGenerator extends PatternGenerator {
+public class PhysicalUnaryPatternGenerator extends PlanPatternGenerator {
 
-    public PhysicalUnaryPatternGenerator(PatternGeneratorAnalyzer analyzer,
+    public PhysicalUnaryPatternGenerator(PlanPatternGeneratorAnalyzer analyzer,
             ClassDeclaration opType, Set<String> parentClass, boolean isMemoPattern) {
         super(analyzer, opType, parentClass, isMemoPattern);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PatternGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGenerator.java
similarity index 96%
rename from fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PatternGenerator.java
rename to fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGenerator.java
index 75c950f8c82..b94c9f489e6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PatternGenerator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGenerator.java
@@ -43,8 +43,8 @@ import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
 /** used to generate pattern by plan. */
-public abstract class PatternGenerator {
-    protected final PatternGeneratorAnalyzer analyzer;
+public abstract class PlanPatternGenerator {
+    protected final JavaAstAnalyzer analyzer;
     protected final ClassDeclaration opType;
     protected final Set<String> parentClass;
     protected final List<EnumFieldPatternInfo> enumFieldPatternInfos;
@@ -52,9 +52,9 @@ public abstract class PatternGenerator {
     protected final boolean isMemoPattern;
 
     /** constructor. */
-    public PatternGenerator(PatternGeneratorAnalyzer analyzer, ClassDeclaration opType,
+    public PlanPatternGenerator(PlanPatternGeneratorAnalyzer analyzer, ClassDeclaration opType,
             Set<String> parentClass, boolean isMemoPattern) {
-        this.analyzer = analyzer;
+        this.analyzer = analyzer.getAnalyzer();
         this.opType = opType;
         this.parentClass = parentClass;
         this.enumFieldPatternInfos = getEnumFieldPatternInfos();
@@ -76,8 +76,8 @@ public abstract class PatternGenerator {
     }
 
     /** generate code by generators and analyzer. */
-    public static String generateCode(String className, String parentClassName, List<PatternGenerator> generators,
-            PatternGeneratorAnalyzer analyzer, boolean isMemoPattern) {
+    public static String generateCode(String className, String parentClassName, List<PlanPatternGenerator> generators,
+            PlanPatternGeneratorAnalyzer analyzer, boolean isMemoPattern) {
         String generateCode
                 = "// Licensed to the Apache Software Foundation (ASF) under one\n"
                 + "// or more contributor license agreements.  See the NOTICE file\n"
@@ -206,7 +206,7 @@ public abstract class PatternGenerator {
     }
 
     /** create generator by plan's type. */
-    public static Optional<PatternGenerator> create(PatternGeneratorAnalyzer analyzer,
+    public static Optional<PlanPatternGenerator> create(PlanPatternGeneratorAnalyzer analyzer,
             ClassDeclaration opType, Set<String> parentClass, boolean isMemoPattern) {
         if (parentClass.contains("org.apache.doris.nereids.trees.plans.logical.LogicalLeaf")) {
             return Optional.of(new LogicalLeafPatternGenerator(analyzer, opType, parentClass, isMemoPattern));
@@ -225,9 +225,9 @@ public abstract class PatternGenerator {
         }
     }
 
-    private static String generateImports(List<PatternGenerator> generators) {
+    private static String generateImports(List<PlanPatternGenerator> generators) {
         Set<String> imports = new HashSet<>();
-        for (PatternGenerator generator : generators) {
+        for (PlanPatternGenerator generator : generators) {
             imports.addAll(generator.getImports());
         }
         List<String> sortedImports = new ArrayList<>(imports);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGeneratorAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGeneratorAnalyzer.java
new file mode 100644
index 00000000000..99d7c308dac
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGeneratorAnalyzer.java
@@ -0,0 +1,73 @@
+// 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.pattern.generator;
+
+import org.apache.doris.nereids.pattern.generator.javaast.ClassDeclaration;
+
+import java.lang.reflect.Modifier;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * used to analyze plan class extends hierarchy and then generated pattern builder methods.
+ */
+public class PlanPatternGeneratorAnalyzer {
+    private final JavaAstAnalyzer analyzer;
+
+    public PlanPatternGeneratorAnalyzer(JavaAstAnalyzer analyzer) {
+        this.analyzer = analyzer;
+    }
+
+    public JavaAstAnalyzer getAnalyzer() {
+        return analyzer;
+    }
+
+    /** generate pattern methods. */
+    public String generatePatterns(String className, String parentClassName, boolean isMemoPattern) {
+        Map<ClassDeclaration, Set<String>> planClassMap = analyzer.getParentClassMap().entrySet().stream()
+                .filter(kv -> kv.getValue().contains("org.apache.doris.nereids.trees.plans.Plan"))
+                .filter(kv -> !kv.getKey().name.equals("GroupPlan"))
+                .filter(kv -> !Modifier.isAbstract(kv.getKey().modifiers.mod)
+                        && kv.getKey() instanceof ClassDeclaration)
+                .collect(Collectors.toMap(kv -> (ClassDeclaration) kv.getKey(), kv -> kv.getValue()));
+
+        List<PlanPatternGenerator> generators = planClassMap.entrySet()
+                .stream()
+                .map(kv -> PlanPatternGenerator.create(this, kv.getKey(), kv.getValue(), isMemoPattern))
+                .filter(Optional::isPresent)
+                .map(Optional::get)
+                .sorted((g1, g2) -> {
+                    // logical first
+                    if (g1.isLogical() != g2.isLogical()) {
+                        return g1.isLogical() ? -1 : 1;
+                    }
+                    // leaf first
+                    if (g1.childrenNum() != g2.childrenNum()) {
+                        return g1.childrenNum() - g2.childrenNum();
+                    }
+                    // string dict sort
+                    return g1.opType.name.compareTo(g2.opType.name);
+                })
+                .collect(Collectors.toList());
+
+        return PlanPatternGenerator.generateCode(className, parentClassName, generators, this, isMemoPattern);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanTypeMappingGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanTypeMappingGenerator.java
new file mode 100644
index 00000000000..c3b6c765d49
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanTypeMappingGenerator.java
@@ -0,0 +1,159 @@
+// 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.pattern.generator;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.stream.Collectors;
+import javax.annotation.processing.ProcessingEnvironment;
+import javax.tools.StandardLocation;
+
+/** PlanTypeMappingGenerator */
+public class PlanTypeMappingGenerator {
+    private final JavaAstAnalyzer analyzer;
+
+    public PlanTypeMappingGenerator(JavaAstAnalyzer javaAstAnalyzer) {
+        this.analyzer = javaAstAnalyzer;
+    }
+
+    public JavaAstAnalyzer getAnalyzer() {
+        return analyzer;
+    }
+
+    /** generate */
+    public void generate(ProcessingEnvironment processingEnv) throws IOException {
+        Set<String> superPlans = findSuperPlan();
+        Map<String, Set<String>> childrenNameMap = analyzer.getChildrenNameMap();
+        Map<String, Set<String>> parentNameMap = analyzer.getParentNameMap();
+        String code = generateCode(childrenNameMap, parentNameMap, superPlans);
+        generateFile(processingEnv, code);
+    }
+
+    private void generateFile(ProcessingEnvironment processingEnv, String code) throws IOException {
+        File generatePatternFile = new File(processingEnv.getFiler()
+                .getResource(StandardLocation.SOURCE_OUTPUT, "org.apache.doris.nereids.pattern",
+                        "GeneratedPlanRelations.java").toUri());
+        if (generatePatternFile.exists()) {
+            generatePatternFile.delete();
+        }
+        if (!generatePatternFile.getParentFile().exists()) {
+            generatePatternFile.getParentFile().mkdirs();
+        }
+
+        // bypass create file for processingEnv.getFiler(), compile GeneratePatterns in next compile term
+        try (BufferedWriter bufferedWriter = new BufferedWriter(new FileWriter(generatePatternFile))) {
+            bufferedWriter.write(code);
+        }
+    }
+
+    private Set<String> findSuperPlan() {
+        Map<String, Set<String>> parentNameMap = analyzer.getParentNameMap();
+        Map<String, Set<String>> childrenNameMap = analyzer.getChildrenNameMap();
+        Set<String> superPlans = Sets.newLinkedHashSet();
+        for (Entry<String, Set<String>> entry : childrenNameMap.entrySet()) {
+            String parentName = entry.getKey();
+            Set<String> childrenNames = entry.getValue();
+
+            if (parentName.startsWith("org.apache.doris.nereids.trees.plans.")) {
+                for (String childrenName : childrenNames) {
+                    Set<String> parentNames = parentNameMap.get(childrenName);
+                    if (parentNames != null
+                            && parentNames.contains("org.apache.doris.nereids.trees.plans.Plan")) {
+                        superPlans.add(parentName);
+                        break;
+                    }
+                }
+            }
+        }
+        return superPlans;
+    }
+
+    private String generateCode(Map<String, Set<String>> childrenNameMap,
+            Map<String, Set<String>> parentNameMap, Set<String> superPlans) {
+        String generateCode
+                = "// Licensed to the Apache Software Foundation (ASF) under one\n"
+                + "// or more contributor license agreements.  See the NOTICE file\n"
+                + "// distributed with this work for additional information\n"
+                + "// regarding copyright ownership.  The ASF licenses this file\n"
+                + "// to you under the Apache License, Version 2.0 (the\n"
+                + "// \"License\"); you may not use this file except in compliance\n"
+                + "// with the License.  You may obtain a copy of the License at\n"
+                + "//\n"
+                + "//   http://www.apache.org/licenses/LICENSE-2.0\n"
+                + "//\n"
+                + "// Unless required by applicable law or agreed to in writing,\n"
+                + "// software distributed under the License is distributed on an\n"
+                + "// \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n"
+                + "// KIND, either express or implied.  See the License for the\n"
+                + "// specific language governing permissions and limitations\n"
+                + "// under the License.\n"
+                + "\n"
+                + "package org.apache.doris.nereids.pattern;\n"
+                + "\n"
+                + "import org.apache.doris.nereids.trees.plans.Plan;\n"
+                + "\n"
+                + "import com.google.common.collect.ImmutableMap;\n"
+                + "import com.google.common.collect.ImmutableSet;\n"
+                + "\n"
+                + "import java.util.Map;\n"
+                + "import java.util.Set;\n"
+                + "\n";
+        generateCode += "/** GeneratedPlanRelations */\npublic class GeneratedPlanRelations {\n";
+        String childrenClassesGenericType = "<Class<?>, Set<Class<? extends Plan>>>";
+        generateCode +=
+                "    public static final Map" + childrenClassesGenericType + " CHILDREN_CLASS_MAP;\n\n";
+        generateCode +=
+                "    static {\n"
+              + "        ImmutableMap.Builder" + childrenClassesGenericType + " childrenClassesBuilder\n"
+              + "                = ImmutableMap.builderWithExpectedSize(" + childrenNameMap.size() + ");\n";
+
+        for (String superPlan : superPlans) {
+            Set<String> childrenClasseSet = childrenNameMap.get(superPlan)
+                    .stream()
+                    .filter(childClass -> parentNameMap.get(childClass)
+                            .contains("org.apache.doris.nereids.trees.plans.Plan")
+                    )
+                    .collect(Collectors.toSet());
+
+            List<String> childrenClasses = Lists.newArrayList(childrenClasseSet);
+            Collections.sort(childrenClasses, Comparator.naturalOrder());
+
+            String childClassesString = childrenClasses.stream()
+                    .map(childClass -> "                    " + childClass + ".class")
+                    .collect(Collectors.joining(",\n"));
+            generateCode += "        childrenClassesBuilder.put(\n                " + superPlan
+                    + ".class,\n                ImmutableSet.<Class<? extends Plan>>of(\n" + childClassesString
+                    + "\n                )\n        );\n\n";
+        }
+
+        generateCode += "        CHILDREN_CLASS_MAP = childrenClassesBuilder.build();\n";
+
+        return generateCode + "    }\n}\n";
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java
index 4efafe3af90..fb6e54e38a8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java
@@ -195,9 +195,20 @@ public class RuntimeFilterPruner extends PlanPostProcessor {
     @Override
     public PhysicalFilter visitPhysicalFilter(PhysicalFilter<? extends Plan> filter, CascadesContext context) {
         filter.child().accept(this, context);
-        boolean visibleFilter = filter.getExpressions().stream()
-                .flatMap(expression -> expression.getInputSlots().stream())
-                .anyMatch(slot -> isVisibleColumn(slot));
+
+        boolean visibleFilter = false;
+
+        for (Expression expr : filter.getExpressions()) {
+            for (Slot inputSlot : expr.getInputSlots()) {
+                if (isVisibleColumn(inputSlot)) {
+                    visibleFilter = true;
+                    break;
+                }
+            }
+            if (visibleFilter) {
+                break;
+            }
+        }
         if (visibleFilter) {
             // skip filters like: __DORIS_DELETE_SIGN__ = 0
             context.getRuntimeFilterContext().addEffectiveSrcNode(filter, RuntimeFilterContext.EffectiveSrcType.NATIVE);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/Validator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/Validator.java
index e73039e9237..561e09ed404 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/Validator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/Validator.java
@@ -26,6 +26,8 @@ import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.algebra.Aggregate;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalProject;
+import org.apache.doris.nereids.util.PlanUtils;
+import org.apache.doris.nereids.util.Utils;
 
 import com.google.common.base.Preconditions;
 
@@ -69,7 +71,10 @@ public class Validator extends PlanPostProcessor {
 
     @Override
     public Plan visit(Plan plan, CascadesContext context) {
-        plan.children().forEach(child -> child.accept(this, context));
+        for (Plan child : plan.children()) {
+            child.accept(this, context);
+        }
+
         Optional<Slot> opt = checkAllSlotFromChildren(plan);
         if (opt.isPresent()) {
             List<Slot> childrenOutput = plan.children().stream().flatMap(p -> p.getOutput().stream()).collect(
@@ -93,8 +98,7 @@ public class Validator extends PlanPostProcessor {
         if (plan instanceof Aggregate) {
             return Optional.empty();
         }
-        Set<Slot> childOutputSet = plan.children().stream().flatMap(child -> child.getOutputSet().stream())
-                .collect(Collectors.toSet());
+        Set<Slot> childOutputSet = Utils.fastToImmutableSet(PlanUtils.fastGetChildrenOutputs(plan.children()));
         Set<Slot> inputSlots = plan.getInputSlots();
         for (Slot slot : inputSlots) {
             if (slot.getName().startsWith("mv") || slot instanceof SlotNotFromChildren) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/FunctionalDependencies.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/FunctionalDependencies.java
index d7b4b3b1c9f..c7e6030e137 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/FunctionalDependencies.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/FunctionalDependencies.java
@@ -20,6 +20,7 @@ package org.apache.doris.nereids.properties;
 import org.apache.doris.nereids.trees.expressions.Slot;
 
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
 
 import java.util.HashSet;
 import java.util.Map;
@@ -196,12 +197,23 @@ public class FunctionalDependencies {
         }
 
         public void removeNotContain(Set<Slot> slotSet) {
-            slots = slots.stream()
-                    .filter(slotSet::contains)
-                    .collect(Collectors.toSet());
-            slotSets = slotSets.stream()
-                    .filter(slotSet::containsAll)
-                    .collect(Collectors.toSet());
+            if (!slotSet.isEmpty()) {
+                Set<Slot> newSlots = Sets.newLinkedHashSetWithExpectedSize(slots.size());
+                for (Slot slot : slots) {
+                    if (slotSet.contains(slot)) {
+                        newSlots.add(slot);
+                    }
+                }
+                this.slots = newSlots;
+
+                Set<ImmutableSet<Slot>> newSlotSets = Sets.newLinkedHashSetWithExpectedSize(slots.size());
+                for (ImmutableSet<Slot> set : slotSets) {
+                    if (slotSet.containsAll(set)) {
+                        newSlotSets.add(set);
+                    }
+                }
+                this.slotSets = newSlotSets;
+            }
         }
 
         public void add(Slot slot) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java
index 07d28828942..ea8a6e29dde 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/LogicalProperties.java
@@ -19,7 +19,6 @@ package org.apache.doris.nereids.properties;
 
 import org.apache.doris.common.Id;
 import org.apache.doris.nereids.trees.expressions.ExprId;
-import org.apache.doris.nereids.trees.expressions.NamedExpression;
 import org.apache.doris.nereids.trees.expressions.Slot;
 
 import com.google.common.base.Supplier;
@@ -62,21 +61,40 @@ public class LogicalProperties {
         this.outputSupplier = Suppliers.memoize(
                 Objects.requireNonNull(outputSupplier, "outputSupplier can not be null")
         );
-        this.outputExprIdsSupplier = Suppliers.memoize(
-                () -> this.outputSupplier.get().stream().map(NamedExpression::getExprId).map(Id.class::cast)
-                        .collect(ImmutableList.toImmutableList())
-        );
-        this.outputSetSupplier = Suppliers.memoize(
-                () -> ImmutableSet.copyOf(this.outputSupplier.get())
-        );
-        this.outputMapSupplier = Suppliers.memoize(
-                () -> this.outputSetSupplier.get().stream().collect(ImmutableMap.toImmutableMap(s -> s, s -> s))
-        );
-        this.outputExprIdSetSupplier = Suppliers.memoize(
-                () -> this.outputSupplier.get().stream()
-                        .map(NamedExpression::getExprId)
-                        .collect(ImmutableSet.toImmutableSet())
-        );
+        this.outputExprIdsSupplier = Suppliers.memoize(() -> {
+            List<Slot> output = this.outputSupplier.get();
+            ImmutableList.Builder<Id> exprIdSet
+                    = ImmutableList.builderWithExpectedSize(output.size());
+            for (Slot slot : output) {
+                exprIdSet.add(slot.getExprId());
+            }
+            return exprIdSet.build();
+        });
+        this.outputSetSupplier = Suppliers.memoize(() -> {
+            List<Slot> output = outputSupplier.get();
+            ImmutableSet.Builder<Slot> slots = ImmutableSet.builderWithExpectedSize(output.size());
+            for (Slot slot : output) {
+                slots.add(slot);
+            }
+            return slots.build();
+        });
+        this.outputMapSupplier = Suppliers.memoize(() -> {
+            Set<Slot> slots = outputSetSupplier.get();
+            ImmutableMap.Builder<Slot, Slot> map = ImmutableMap.builderWithExpectedSize(slots.size());
+            for (Slot slot : slots) {
+                map.put(slot, slot);
+            }
+            return map.build();
+        });
+        this.outputExprIdSetSupplier = Suppliers.memoize(() -> {
+            List<Slot> output = this.outputSupplier.get();
+            ImmutableSet.Builder<ExprId> exprIdSet
+                    = ImmutableSet.builderWithExpectedSize(output.size());
+            for (Slot slot : output) {
+                exprIdSet.add(slot.getExprId());
+            }
+            return exprIdSet.build();
+        });
         this.fdSupplier = Suppliers.memoize(
                 Objects.requireNonNull(fdSupplier, "FunctionalDependencies can not be null")
         );
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java
index a9b4591ad4a..207dd6458c9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java
@@ -24,8 +24,8 @@ import org.apache.doris.nereids.pattern.Pattern;
 import org.apache.doris.nereids.rules.RuleType.RuleTypeClass;
 import org.apache.doris.nereids.trees.plans.Plan;
 
+import java.util.BitSet;
 import java.util.List;
-import java.util.Set;
 
 /**
  * Abstract class for all rules.
@@ -79,8 +79,8 @@ public abstract class Rule {
     /**
      * Filter out already applied rules and rules that are not matched on root node.
      */
-    public boolean isInvalid(Set<Integer> disableRules, GroupExpression groupExpression) {
-        return disableRules.contains(this.getRuleType().type())
+    public boolean isInvalid(BitSet disableRules, GroupExpression groupExpression) {
+        return disableRules.get(this.getRuleType().type())
                 || !groupExpression.notApplied(this)
                 || !this.getPattern().matchRoot(groupExpression.getPlan());
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java
index 408b0d7355e..d317b1e8738 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java
@@ -49,6 +49,7 @@ import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectAggr
 import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectFilterAggregateRule;
 import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectFilterJoinRule;
 import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectJoinRule;
+import org.apache.doris.nereids.rules.expression.ExpressionOptimization;
 import org.apache.doris.nereids.rules.implementation.AggregateStrategies;
 import org.apache.doris.nereids.rules.implementation.LogicalAssertNumRowsToPhysicalAssertNumRows;
 import org.apache.doris.nereids.rules.implementation.LogicalCTEAnchorToPhysicalCTEAnchor;
@@ -153,7 +154,8 @@ public class RuleSet {
             new MergeLimits(),
             new PushDownAliasThroughJoin(),
             new PushDownFilterThroughWindow(),
-            new PushDownFilterThroughPartitionTopN()
+            new PushDownFilterThroughPartitionTopN(),
+            new ExpressionOptimization()
     );
 
     public static final List<Rule> IMPLEMENTATION_RULES = planRuleFactories()
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AdjustAggregateNullableForEmptySet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AdjustAggregateNullableForEmptySet.java
index 86a70d35ccc..5543341ae27 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AdjustAggregateNullableForEmptySet.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AdjustAggregateNullableForEmptySet.java
@@ -46,21 +46,30 @@ public class AdjustAggregateNullableForEmptySet implements RewriteRuleFactory {
                 RuleType.ADJUST_NULLABLE_FOR_AGGREGATE_SLOT.build(
                         logicalAggregate()
                                 .then(agg -> {
-                                    List<NamedExpression> output = agg.getOutputExpressions().stream()
-                                            .map(ne -> ((NamedExpression) FunctionReplacer.INSTANCE.replace(ne,
-                                                    agg.getGroupByExpressions().isEmpty())))
-                                            .collect(ImmutableList.toImmutableList());
-                                    return agg.withAggOutput(output);
+                                    List<NamedExpression> outputExprs = agg.getOutputExpressions();
+                                    boolean noGroupBy = agg.getGroupByExpressions().isEmpty();
+                                    ImmutableList.Builder<NamedExpression> newOutput
+                                            = ImmutableList.builderWithExpectedSize(outputExprs.size());
+                                    for (NamedExpression ne : outputExprs) {
+                                        NamedExpression newExpr =
+                                                ((NamedExpression) FunctionReplacer.INSTANCE.replace(ne, noGroupBy));
+                                        newOutput.add(newExpr);
+                                    }
+                                    return agg.withAggOutput(newOutput.build());
                                 })
                 ),
                 RuleType.ADJUST_NULLABLE_FOR_HAVING_SLOT.build(
                         logicalHaving(logicalAggregate())
                                 .then(having -> {
-                                    Set<Expression> newConjuncts = having.getConjuncts().stream()
-                                            .map(ne -> FunctionReplacer.INSTANCE.replace(ne,
-                                                    having.child().getGroupByExpressions().isEmpty()))
-                                            .collect(ImmutableSet.toImmutableSet());
-                                    return new LogicalHaving<>(newConjuncts, having.child());
+                                    Set<Expression> conjuncts = having.getConjuncts();
+                                    boolean noGroupBy = having.child().getGroupByExpressions().isEmpty();
+                                    ImmutableSet.Builder<Expression> newConjuncts
+                                            = ImmutableSet.builderWithExpectedSize(conjuncts.size());
+                                    for (Expression expr : conjuncts) {
+                                        Expression newExpr = FunctionReplacer.INSTANCE.replace(expr, noGroupBy);
+                                        newConjuncts.add(newExpr);
+                                    }
+                                    return new LogicalHaving<>(newConjuncts.build(), having.child());
                                 })
                 )
         );
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java
index c2c7f5815d9..6211f493eaf 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java
@@ -333,10 +333,11 @@ public class BindExpression implements AnalysisRuleFactory {
         List<LogicalPlan> relations
                 = Lists.newArrayListWithCapacity(logicalInlineTable.getConstantExprsList().size());
         for (int i = 0; i < logicalInlineTable.getConstantExprsList().size(); i++) {
-            if (logicalInlineTable.getConstantExprsList().get(i).stream()
-                    .anyMatch(DefaultValueSlot.class::isInstance)) {
-                throw new AnalysisException("Default expression"
-                        + " can't exist in SELECT statement at row " + (i + 1));
+            for (NamedExpression constantExpr : logicalInlineTable.getConstantExprsList().get(i)) {
+                if (constantExpr instanceof DefaultValueSlot) {
+                    throw new AnalysisException("Default expression"
+                            + " can't exist in SELECT statement at row " + (i + 1));
+                }
             }
             relations.add(new UnboundOneRowRelation(StatementScopeIdGenerator.newRelationId(),
                     logicalInlineTable.getConstantExprsList().get(i)));
@@ -590,7 +591,7 @@ public class BindExpression implements AnalysisRuleFactory {
         SimpleExprAnalyzer analyzer = buildSimpleExprAnalyzer(
                 filter, cascadesContext, filter.children(), true, true);
         ImmutableSet.Builder<Expression> boundConjuncts = ImmutableSet.builderWithExpectedSize(
-                filter.getConjuncts().size() * 2);
+                filter.getConjuncts().size());
         for (Expression conjunct : filter.getConjuncts()) {
             Expression boundConjunct = analyzer.analyze(conjunct);
             boundConjunct = TypeCoercionUtils.castIfNotSameType(boundConjunct, BooleanType.INSTANCE);
@@ -828,15 +829,22 @@ public class BindExpression implements AnalysisRuleFactory {
         if (output.stream().noneMatch(Alias.class::isInstance)) {
             return;
         }
-        List<Alias> aliasList = output.stream().filter(Alias.class::isInstance)
-                .map(Alias.class::cast).collect(Collectors.toList());
+        List<Alias> aliasList = ExpressionUtils.filter(output, Alias.class);
 
         List<NamedExpression> exprAliasList =
                 ExpressionUtils.collectAll(expressions, NamedExpression.class::isInstance);
 
-        boolean isGroupByContainAlias = exprAliasList.stream().anyMatch(ne ->
-                aliasList.stream().anyMatch(alias -> !alias.getExprId().equals(ne.getExprId())
-                        && alias.getName().equals(ne.getName())));
+        boolean isGroupByContainAlias = false;
+        for (NamedExpression ne : exprAliasList) {
+            for (Alias alias : aliasList) {
+                if (!alias.getExprId().equals(ne.getExprId()) && alias.getName().equalsIgnoreCase(ne.getName())) {
+                    isGroupByContainAlias = true;
+                }
+            }
+            if (isGroupByContainAlias) {
+                break;
+            }
+        }
 
         if (isGroupByContainAlias
                 && ConnectContext.get() != null
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotWithPaths.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotWithPaths.java
index 114e4c1d120..714e6e48794 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotWithPaths.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSlotWithPaths.java
@@ -22,7 +22,6 @@ import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.rules.RuleType;
 import org.apache.doris.nereids.trees.expressions.Alias;
 import org.apache.doris.nereids.trees.expressions.NamedExpression;
-import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.expressions.SlotReference;
 import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
@@ -34,7 +33,6 @@ import com.google.common.collect.ImmutableList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * Rule to bind slot with path in query plan.
@@ -60,21 +58,18 @@ public class BindSlotWithPaths implements AnalysisRuleFactory {
                             Set<SlotReference> pathsSlots = ctx.statementContext.getAllPathsSlots();
                             // With new logical properties that contains new slots with paths
                             StatementContext stmtCtx = ConnectContext.get().getStatementContext();
-                            List<Slot> olapScanPathSlots = pathsSlots.stream().filter(
-                                    slot -> {
-                                        Preconditions.checkNotNull(stmtCtx.getRelationBySlot(slot),
-                                                "[Not implemented] Slot not found in relation map, slot ", slot);
-                                        return stmtCtx.getRelationBySlot(slot).getRelationId()
-                                                == logicalOlapScan.getRelationId();
-                                    }).collect(
-                                    Collectors.toList());
-                            List<NamedExpression> newExprs = olapScanPathSlots.stream()
-                                    .map(SlotReference.class::cast)
-                                    .map(slotReference ->
-                                            new Alias(slotReference.getExprId(),
-                                                    stmtCtx.getOriginalExpr(slotReference), slotReference.getName()))
-                                    .collect(
-                                            Collectors.toList());
+                            ImmutableList.Builder<NamedExpression> newExprsBuilder
+                                    = ImmutableList.builderWithExpectedSize(pathsSlots.size());
+                            for (SlotReference slot : pathsSlots) {
+                                Preconditions.checkNotNull(stmtCtx.getRelationBySlot(slot),
+                                        "[Not implemented] Slot not found in relation map, slot ", slot);
+                                if (stmtCtx.getRelationBySlot(slot).getRelationId()
+                                        == logicalOlapScan.getRelationId()) {
+                                    newExprsBuilder.add(new Alias(slot.getExprId(),
+                                            stmtCtx.getOriginalExpr(slot), slot.getName()));
+                                }
+                            }
+                            ImmutableList<NamedExpression> newExprs = newExprsBuilder.build();
                             if (newExprs.isEmpty()) {
                                 return ctx.root;
                             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java
index 754d3efa583..92052bc85ed 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java
@@ -46,6 +46,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
 import org.apache.doris.nereids.trees.plans.logical.LogicalTopN;
 import org.apache.doris.nereids.trees.plans.logical.LogicalWindow;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.commons.lang3.StringUtils;
 
 import java.util.List;
@@ -69,42 +70,43 @@ public class CheckAfterRewrite extends OneAnalysisRuleFactory {
     }
 
     private void checkUnexpectedExpression(Plan plan) {
-        if (plan.getExpressions().stream().anyMatch(e -> e.anyMatch(SubqueryExpr.class::isInstance))) {
-            throw new AnalysisException("Subquery is not allowed in " + plan.getType());
-        }
-        if (!(plan instanceof Generate)) {
-            if (plan.getExpressions().stream().anyMatch(e -> e.anyMatch(TableGeneratingFunction.class::isInstance))) {
-                throw new AnalysisException("table generating function is not allowed in " + plan.getType());
-            }
+        boolean isGenerate = plan instanceof Generate;
+        boolean isAgg = plan instanceof LogicalAggregate;
+        boolean isWindow = plan instanceof LogicalWindow;
+        boolean notAggAndWindow = !isAgg && !isWindow;
+
+        for (Expression expression : plan.getExpressions()) {
+            expression.foreach(expr -> {
+                if (expr instanceof SubqueryExpr) {
+                    throw new AnalysisException("Subquery is not allowed in " + plan.getType());
+                } else if (!isGenerate && expr instanceof TableGeneratingFunction) {
+                    throw new AnalysisException("table generating function is not allowed in " + plan.getType());
+                } else if (notAggAndWindow && expr instanceof AggregateFunction) {
+                    throw new AnalysisException("aggregate function is not allowed in " + plan.getType());
+                } else if (!isAgg && expr instanceof GroupingScalarFunction) {
+                    throw new AnalysisException("grouping scalar function is not allowed in " + plan.getType());
+                } else if (!isWindow && expr instanceof WindowExpression) {
+                    throw new AnalysisException("analytic function is not allowed in " + plan.getType());
+                }
+            });
         }
-        if (!(plan instanceof LogicalAggregate || plan instanceof LogicalWindow)) {
-            if (plan.getExpressions().stream().anyMatch(e -> e.anyMatch(AggregateFunction.class::isInstance))) {
-                throw new AnalysisException("aggregate function is not allowed in " + plan.getType());
+    }
+
+    private void checkAllSlotReferenceFromChildren(Plan plan) {
+        Set<Slot> inputSlots = plan.getInputSlots();
+        Set<ExprId> childrenOutput = plan.getChildrenOutputExprIdSet();
+
+        ImmutableSet.Builder<Slot> notFromChildrenBuilder = ImmutableSet.builderWithExpectedSize(inputSlots.size());
+        for (Slot inputSlot : inputSlots) {
+            if (!childrenOutput.contains(inputSlot.getExprId())) {
+                notFromChildrenBuilder.add(inputSlot);
             }
         }
-        if (!(plan instanceof LogicalAggregate)) {
-            if (plan.getExpressions().stream().anyMatch(e -> e.anyMatch(GroupingScalarFunction.class::isInstance))) {
-                throw new AnalysisException("grouping scalar function is not allowed in " + plan.getType());
-            }
+        Set<Slot> notFromChildren = notFromChildrenBuilder.build();
+        if (notFromChildren.isEmpty()) {
+            return;
         }
-        if (!(plan instanceof LogicalWindow)) {
-            if (plan.getExpressions().stream().anyMatch(e -> e.anyMatch(WindowExpression.class::isInstance))) {
-                throw new AnalysisException("analytic function is not allowed in " + plan.getType());
-            }
-        }
-    }
 
-    private void checkAllSlotReferenceFromChildren(Plan plan) {
-        Set<Slot> notFromChildren = plan.getExpressions().stream()
-                .flatMap(expr -> expr.getInputSlots().stream())
-                .collect(Collectors.toSet());
-        Set<ExprId> childrenOutput = plan.children().stream()
-                .flatMap(child -> child.getOutput().stream())
-                .map(NamedExpression::getExprId)
-                .collect(Collectors.toSet());
-        notFromChildren = notFromChildren.stream()
-                .filter(s -> !childrenOutput.contains(s.getExprId()))
-                .collect(Collectors.toSet());
         notFromChildren = removeValidSlotsNotFromChildren(notFromChildren, childrenOutput);
         if (!notFromChildren.isEmpty()) {
             if (plan.arity() != 0 && plan.child(0) instanceof LogicalAggregate) {
@@ -181,17 +183,18 @@ public class CheckAfterRewrite extends OneAnalysisRuleFactory {
     }
 
     private void checkMatchIsUsedCorrectly(Plan plan) {
-        if (plan.getExpressions().stream().anyMatch(
-                expression -> expression instanceof Match)) {
-            if (plan instanceof LogicalFilter && (plan.child(0) instanceof LogicalOlapScan
-                    || plan.child(0) instanceof LogicalDeferMaterializeOlapScan
-                    || plan.child(0) instanceof LogicalProject
+        for (Expression expression : plan.getExpressions()) {
+            if (expression instanceof Match) {
+                if (plan instanceof LogicalFilter && (plan.child(0) instanceof LogicalOlapScan
+                        || plan.child(0) instanceof LogicalDeferMaterializeOlapScan
+                        || plan.child(0) instanceof LogicalProject
                         && ((LogicalProject<?>) plan.child(0)).hasPushedDownToProjectionFunctions())) {
-                return;
-            } else {
-                throw new AnalysisException(String.format(
-                    "Not support match in %s in plan: %s, only support in olapScan filter",
-                    plan.child(0), plan));
+                    return;
+                } else {
+                    throw new AnalysisException(String.format(
+                            "Not support match in %s in plan: %s, only support in olapScan filter",
+                            plan.child(0), plan));
+                }
             }
         }
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java
index 5a310d697ac..64fd14019bb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java
@@ -45,7 +45,6 @@ import com.google.common.collect.ImmutableSet;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Set;
 
 /**
@@ -117,14 +116,16 @@ public class CheckAnalysis implements AnalysisRuleFactory {
         if (unexpectedExpressionTypes.isEmpty()) {
             return;
         }
-        plan.getExpressions().forEach(c -> c.foreachUp(e -> {
-            for (Class<? extends Expression> type : unexpectedExpressionTypes) {
-                if (type.isInstance(e)) {
-                    throw new AnalysisException(plan.getType() + " can not contains "
-                            + type.getSimpleName() + " expression: " + ((Expression) e).toSql());
+        for (Expression expr : plan.getExpressions()) {
+            expr.foreachUp(e -> {
+                for (Class<? extends Expression> type : unexpectedExpressionTypes) {
+                    if (type.isInstance(e)) {
+                        throw new AnalysisException(plan.getType() + " can not contains "
+                                + type.getSimpleName() + " expression: " + ((Expression) e).toSql());
+                    }
                 }
-            }
-        }));
+            });
+        }
     }
 
     private void checkExpressionInputTypes(Plan plan) {
@@ -157,20 +158,21 @@ public class CheckAnalysis implements AnalysisRuleFactory {
                 break;
             }
         }
-        long distinctFunctionNum = aggregateFunctions.stream()
-                .filter(AggregateFunction::isDistinct)
-                .count();
+
+        long distinctFunctionNum = 0;
+        for (AggregateFunction aggregateFunction : aggregateFunctions) {
+            distinctFunctionNum += aggregateFunction.isDistinct() ? 1 : 0;
+        }
 
         if (distinctMultiColumns && distinctFunctionNum > 1) {
             throw new AnalysisException(
                     "The query contains multi count distinct or sum distinct, each can't have multi columns");
         }
-        Optional<Expression> expr = aggregate.getGroupByExpressions().stream()
-                .filter(expression -> expression.containsType(AggregateFunction.class)).findFirst();
-        if (expr.isPresent()) {
-            throw new AnalysisException(
-                    "GROUP BY expression must not contain aggregate functions: "
-                            + expr.get().toSql());
+        for (Expression expr : aggregate.getGroupByExpressions()) {
+            if (expr.anyMatch(AggregateFunction.class::isInstance)) {
+                throw new AnalysisException(
+                        "GROUP BY expression must not contain aggregate functions: " + expr.toSql());
+            }
         }
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java
index 4408e64487c..7f7e229da31 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateGroupByConstant.java
@@ -64,7 +64,7 @@ public class EliminateGroupByConstant extends OneRewriteRuleFactory {
                 //   because we rely on expression matching to replace subtree that same as group by expr in output
                 //   if we do constant folding before normalize aggregate, the subtree will change and matching fail
                 //   such as: select a + 1 + 2 + 3, sum(b) from t group by a + 1 + 2
-                Expression foldExpression = FoldConstantRule.INSTANCE.rewrite(expression, context);
+                Expression foldExpression = FoldConstantRule.evaluate(expression, context);
                 if (!foldExpression.isConstant()) {
                     slotGroupByExprs.add(expression);
                 } else {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java
index f4c1b428d41..56ca1b3a8c4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java
@@ -297,7 +297,7 @@ public class ExpressionAnalyzer extends SubExprAnalyzer<ExpressionRewriteContext
         if (unboundFunction.isHighOrder()) {
             unboundFunction = bindHighOrderFunction(unboundFunction, context);
         } else {
-            unboundFunction = (UnboundFunction) rewriteChildren(this, unboundFunction, context);
+            unboundFunction = (UnboundFunction) super.visit(unboundFunction, context);
         }
 
         // bind function
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlots.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlots.java
index d6c783bbe94..82468978a80 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlots.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlots.java
@@ -316,13 +316,18 @@ public class FillUpMissingSlots implements AnalysisRuleFactory {
     }
 
     private boolean checkSort(LogicalSort<? extends Plan> logicalSort) {
-        return logicalSort.getOrderKeys().stream()
-                .map(OrderKey::getExpr)
-                .map(Expression::getInputSlots)
-                .flatMap(Set::stream)
-                .anyMatch(s -> !logicalSort.child().getOutputSet().contains(s))
-                || logicalSort.getOrderKeys().stream()
-                .map(OrderKey::getExpr)
-                .anyMatch(e -> e.containsType(AggregateFunction.class));
+        Plan child = logicalSort.child();
+        for (OrderKey orderKey : logicalSort.getOrderKeys()) {
+            Expression expr = orderKey.getExpr();
+            if (expr.anyMatch(AggregateFunction.class::isInstance)) {
+                return true;
+            }
+            for (Slot inputSlot : expr.getInputSlots()) {
+                if (!child.getOutputSet().contains(inputSlot)) {
+                    return true;
+                }
+            }
+        }
+        return false;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java
index 6f668611b9c..7f6df51248e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java
@@ -38,6 +38,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 import org.apache.doris.nereids.util.ExpressionUtils;
 import org.apache.doris.nereids.util.PlanUtils.CollectNonWindowedAggFuncs;
+import org.apache.doris.nereids.util.Utils;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList.Builder;
@@ -139,8 +140,7 @@ public class NormalizeAggregate implements RewriteRuleFactory, NormalizeToSlot {
 
         // Push down exprs:
         // collect group by exprs
-        Set<Expression> groupingByExprs =
-                ImmutableSet.copyOf(aggregate.getGroupByExpressions());
+        Set<Expression> groupingByExprs = Utils.fastToImmutableSet(aggregate.getGroupByExpressions());
 
         // collect all trivial-agg
         List<NamedExpression> aggregateOutput = aggregate.getOutputExpressions();
@@ -149,27 +149,31 @@ public class NormalizeAggregate implements RewriteRuleFactory, NormalizeToSlot {
         // split non-distinct agg child as two part
         // TRUE part 1: need push down itself, if it contains subquery or window expression
         // FALSE part 2: need push down its input slots, if it DOES NOT contain subquery or window expression
-        Map<Boolean, Set<Expression>> categorizedNoDistinctAggsChildren = aggFuncs.stream()
+        Map<Boolean, ImmutableSet<Expression>> categorizedNoDistinctAggsChildren = aggFuncs.stream()
                 .filter(aggFunc -> !aggFunc.isDistinct())
                 .flatMap(agg -> agg.children().stream())
                 .collect(Collectors.groupingBy(
                         child -> child.containsType(SubqueryExpr.class, WindowExpression.class),
-                        Collectors.toSet()));
+                        ImmutableSet.toImmutableSet()));
 
         // split distinct agg child as two parts
         // TRUE part 1: need push down itself, if it is NOT SlotReference or Literal
         // FALSE part 2: need push down its input slots, if it is SlotReference or Literal
-        Map<Boolean, Set<Expression>> categorizedDistinctAggsChildren = aggFuncs.stream()
+        Map<Object, ImmutableSet<Expression>> categorizedDistinctAggsChildren = aggFuncs.stream()
                 .filter(AggregateFunction::isDistinct)
                 .flatMap(agg -> agg.children().stream())
-                .collect(Collectors.groupingBy(child -> !(child instanceof SlotReference), Collectors.toSet()));
+                .collect(
+                        Collectors.groupingBy(
+                                child -> !(child instanceof SlotReference),
+                                ImmutableSet.toImmutableSet())
+                );
 
         Set<Expression> needPushSelf = Sets.union(
-                categorizedNoDistinctAggsChildren.getOrDefault(true, new HashSet<>()),
-                categorizedDistinctAggsChildren.getOrDefault(true, new HashSet<>()));
+                categorizedNoDistinctAggsChildren.getOrDefault(true, ImmutableSet.of()),
+                categorizedDistinctAggsChildren.getOrDefault(true, ImmutableSet.of()));
         Set<Slot> needPushInputSlots = ExpressionUtils.getInputSlotSet(Sets.union(
-                categorizedNoDistinctAggsChildren.getOrDefault(false, new HashSet<>()),
-                categorizedDistinctAggsChildren.getOrDefault(false, new HashSet<>())));
+                categorizedNoDistinctAggsChildren.getOrDefault(false, ImmutableSet.of()),
+                categorizedDistinctAggsChildren.getOrDefault(false, ImmutableSet.of())));
 
         Set<Alias> existsAlias =
                 ExpressionUtils.mutableCollect(aggregateOutput, Alias.class::isInstance);
@@ -194,8 +198,7 @@ public class NormalizeAggregate implements RewriteRuleFactory, NormalizeToSlot {
         // create bottom project
         Plan bottomPlan;
         if (!bottomProjects.isEmpty()) {
-            bottomPlan = new LogicalProject<>(ImmutableList.copyOf(bottomProjects),
-                    aggregate.child());
+            bottomPlan = new LogicalProject<>(ImmutableList.copyOf(bottomProjects), aggregate.child());
         } else {
             bottomPlan = aggregate.child();
         }
@@ -230,13 +233,17 @@ public class NormalizeAggregate implements RewriteRuleFactory, NormalizeToSlot {
 
         // agg output include 2 parts
         // pushedGroupByExprs and normalized agg functions
-        List<NamedExpression> normalizedAggOutput = ImmutableList.<NamedExpression>builder()
-                .addAll(pushedGroupByExprs.stream().map(NamedExpression::toSlot).iterator())
-                .addAll(normalizedAggFuncsToSlotContext
-                        .pushDownToNamedExpression(normalizedAggFuncs))
-                .build();
 
+        ImmutableList.Builder<NamedExpression> normalizedAggOutputBuilder
+                = ImmutableList.builderWithExpectedSize(groupingByExprs.size() + normalizedAggFuncs.size());
+        for (NamedExpression pushedGroupByExpr : pushedGroupByExprs) {
+            normalizedAggOutputBuilder.add(pushedGroupByExpr.toSlot());
+        }
+        normalizedAggOutputBuilder.addAll(
+                normalizedAggFuncsToSlotContext.pushDownToNamedExpression(normalizedAggFuncs)
+        );
         // create new agg node
+        ImmutableList<NamedExpression> normalizedAggOutput = normalizedAggOutputBuilder.build();
         LogicalAggregate<?> newAggregate =
                 aggregate.withNormalized(normalizedGroupExprs, normalizedAggOutput, bottomPlan);
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ReplaceExpressionByChildOutput.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ReplaceExpressionByChildOutput.java
index b52e2f0218d..cd53086f966 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ReplaceExpressionByChildOutput.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ReplaceExpressionByChildOutput.java
@@ -22,6 +22,7 @@ import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.rules.RuleType;
 import org.apache.doris.nereids.trees.expressions.Alias;
 import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
 import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
@@ -35,7 +36,6 @@ import com.google.common.collect.Maps;
 
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * replace.
@@ -47,52 +47,50 @@ public class ReplaceExpressionByChildOutput implements AnalysisRuleFactory {
                 .add(RuleType.REPLACE_SORT_EXPRESSION_BY_CHILD_OUTPUT.build(
                         logicalSort(logicalProject()).then(sort -> {
                             LogicalProject<Plan> project = sort.child();
-                            Map<Expression, Slot> sMap = Maps.newHashMap();
-                            project.getProjects().stream()
-                                    .filter(Alias.class::isInstance)
-                                    .map(Alias.class::cast)
-                                    .forEach(p -> sMap.put(p.child(), p.toSlot()));
+                            Map<Expression, Slot> sMap = buildOutputAliasMap(project.getProjects());
                             return replaceSortExpression(sort, sMap);
                         })
                 ))
                 .add(RuleType.REPLACE_SORT_EXPRESSION_BY_CHILD_OUTPUT.build(
                         logicalSort(logicalAggregate()).then(sort -> {
                             LogicalAggregate<Plan> aggregate = sort.child();
-                            Map<Expression, Slot> sMap = Maps.newHashMap();
-                            aggregate.getOutputExpressions().stream()
-                                    .filter(Alias.class::isInstance)
-                                    .map(Alias.class::cast)
-                                    .forEach(p -> sMap.put(p.child(), p.toSlot()));
+                            Map<Expression, Slot> sMap = buildOutputAliasMap(aggregate.getOutputExpressions());
                             return replaceSortExpression(sort, sMap);
                         })
                 )).add(RuleType.REPLACE_SORT_EXPRESSION_BY_CHILD_OUTPUT.build(
                         logicalSort(logicalHaving(logicalAggregate())).then(sort -> {
                             LogicalAggregate<Plan> aggregate = sort.child().child();
-                            Map<Expression, Slot> sMap = Maps.newHashMap();
-                            aggregate.getOutputExpressions().stream()
-                                    .filter(Alias.class::isInstance)
-                                    .map(Alias.class::cast)
-                                    .forEach(p -> sMap.put(p.child(), p.toSlot()));
+                            Map<Expression, Slot> sMap = buildOutputAliasMap(aggregate.getOutputExpressions());
                             return replaceSortExpression(sort, sMap);
                         })
                 ))
                 .build();
     }
 
+    private Map<Expression, Slot> buildOutputAliasMap(List<NamedExpression> output) {
+        Map<Expression, Slot> sMap = Maps.newHashMapWithExpectedSize(output.size());
+        for (NamedExpression expr : output) {
+            if (expr instanceof Alias) {
+                Alias alias = (Alias) expr;
+                sMap.put(alias.child(), alias.toSlot());
+            }
+        }
+        return sMap;
+    }
+
     private LogicalPlan replaceSortExpression(LogicalSort<? extends LogicalPlan> sort, Map<Expression, Slot> sMap) {
         List<OrderKey> orderKeys = sort.getOrderKeys();
-        AtomicBoolean changed = new AtomicBoolean(false);
-        List<OrderKey> newKeys = orderKeys.stream().map(k -> {
+
+        boolean changed = false;
+        ImmutableList.Builder<OrderKey> newKeys = ImmutableList.builderWithExpectedSize(orderKeys.size());
+        for (OrderKey k : orderKeys) {
             Expression newExpr = ExpressionUtils.replace(k.getExpr(), sMap);
             if (newExpr != k.getExpr()) {
-                changed.set(true);
+                changed = true;
             }
-            return new OrderKey(newExpr, k.isAsc(), k.isNullFirst());
-        }).collect(ImmutableList.toImmutableList());
-        if (changed.get()) {
-            return new LogicalSort<>(newKeys, sort.child());
-        } else {
-            return sort;
+            newKeys.add(new OrderKey(newExpr, k.isAsc(), k.isNullFirst()));
         }
+
+        return changed ? new LogicalSort<>(newKeys.build(), sort.child()) : sort;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java
index b0f78be54a2..cfc5b2ba24a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubqueryToApply.java
@@ -21,6 +21,7 @@ import org.apache.doris.nereids.CascadesContext;
 import org.apache.doris.nereids.StatementContext;
 import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
 import org.apache.doris.nereids.rules.expression.rules.TrySimplifyPredicateWithMarkJoinSlot;
 import org.apache.doris.nereids.trees.TreeNode;
 import org.apache.doris.nereids.trees.expressions.Alias;
@@ -51,6 +52,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
 import org.apache.doris.nereids.util.ExpressionUtils;
+import org.apache.doris.nereids.util.Utils;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
@@ -77,24 +79,21 @@ public class SubqueryToApply implements AnalysisRuleFactory {
             RuleType.FILTER_SUBQUERY_TO_APPLY.build(
                 logicalFilter().thenApply(ctx -> {
                     LogicalFilter<Plan> filter = ctx.root;
-                    ImmutableList<Set<SubqueryExpr>> subqueryExprsList = filter.getConjuncts().stream()
-                            .<Set<SubqueryExpr>>map(e -> e.collect(SubqueryToApply::canConvertToSupply))
-                            .collect(ImmutableList.toImmutableList());
-                    if (subqueryExprsList.stream()
-                            .flatMap(Collection::stream).noneMatch(SubqueryExpr.class::isInstance)) {
+
+                    Set<Expression> conjuncts = filter.getConjuncts();
+                    CollectSubquerys collectSubquerys = collectSubquerys(conjuncts);
+                    if (!collectSubquerys.hasSubquery) {
                         return filter;
                     }
-                    ImmutableList<Boolean> shouldOutputMarkJoinSlot =
-                            filter.getConjuncts().stream()
-                                    .map(expr -> !(expr instanceof SubqueryExpr)
-                                            && expr.containsType(SubqueryExpr.class))
-                                    .collect(ImmutableList.toImmutableList());
 
-                    List<Expression> oldConjuncts = ImmutableList.copyOf(filter.getConjuncts());
-                    ImmutableList.Builder<Expression> newConjuncts = new ImmutableList.Builder<>();
+                    List<Boolean> shouldOutputMarkJoinSlot = shouldOutputMarkJoinSlot(conjuncts);
+
+                    List<Expression> oldConjuncts = Utils.fastToImmutableList(conjuncts);
+                    ImmutableSet.Builder<Expression> newConjuncts = new ImmutableSet.Builder<>();
                     LogicalPlan applyPlan = null;
                     LogicalPlan tmpPlan = (LogicalPlan) filter.child();
 
+                    List<Set<SubqueryExpr>> subqueryExprsList = collectSubquerys.subqueies;
                     // Subquery traversal with the conjunct of and as the granularity.
                     for (int i = 0; i < subqueryExprsList.size(); ++i) {
                         Set<SubqueryExpr> subqueryExprs = subqueryExprsList.get(i);
@@ -119,9 +118,11 @@ public class SubqueryToApply implements AnalysisRuleFactory {
                         * if it's semi join with non-null mark slot
                         * we can safely change the mark conjunct to hash conjunct
                         */
+                        ExpressionRewriteContext rewriteContext = new ExpressionRewriteContext(ctx.cascadesContext);
                         boolean isMarkSlotNotNull = conjunct.containsType(MarkJoinSlotReference.class)
                                         ? ExpressionUtils.canInferNotNullForMarkSlot(
-                                                TrySimplifyPredicateWithMarkJoinSlot.INSTANCE.rewrite(conjunct, null))
+                                                TrySimplifyPredicateWithMarkJoinSlot.INSTANCE.rewrite(conjunct,
+                                                        rewriteContext), rewriteContext)
                                         : false;
 
                         applyPlan = subqueryToApply(subqueryExprs.stream()
@@ -132,21 +133,22 @@ public class SubqueryToApply implements AnalysisRuleFactory {
                         tmpPlan = applyPlan;
                         newConjuncts.add(conjunct);
                     }
-                    Set<Expression> conjuncts = ImmutableSet.copyOf(newConjuncts.build());
-                    Plan newFilter = new LogicalFilter<>(conjuncts, applyPlan);
+                    Plan newFilter = new LogicalFilter<>(newConjuncts.build(), applyPlan);
                     return new LogicalProject<>(filter.getOutput().stream().collect(ImmutableList.toImmutableList()),
                         newFilter);
                 })
             ),
             RuleType.PROJECT_SUBQUERY_TO_APPLY.build(logicalProject().thenApply(ctx -> {
                 LogicalProject<Plan> project = ctx.root;
-                ImmutableList<Set<SubqueryExpr>> subqueryExprsList = project.getProjects().stream()
-                        .<Set<SubqueryExpr>>map(e -> e.collect(SubqueryToApply::canConvertToSupply))
-                        .collect(ImmutableList.toImmutableList());
-                if (subqueryExprsList.stream().flatMap(Collection::stream).count() == 0) {
+
+                List<NamedExpression> projects = project.getProjects();
+                CollectSubquerys collectSubquerys = collectSubquerys(projects);
+                if (!collectSubquerys.hasSubquery) {
                     return project;
                 }
-                List<NamedExpression> oldProjects = ImmutableList.copyOf(project.getProjects());
+
+                List<Set<SubqueryExpr>> subqueryExprsList = collectSubquerys.subqueies;
+                List<NamedExpression> oldProjects = ImmutableList.copyOf(projects);
                 ImmutableList.Builder<NamedExpression> newProjects = new ImmutableList.Builder<>();
                 LogicalPlan childPlan = (LogicalPlan) project.child();
                 LogicalPlan applyPlan;
@@ -166,7 +168,7 @@ public class SubqueryToApply implements AnalysisRuleFactory {
                             replaceSubquery.replace(oldProjects.get(i), context);
 
                     applyPlan = subqueryToApply(
-                            subqueryExprs.stream().collect(ImmutableList.toImmutableList()),
+                            Utils.fastToImmutableList(subqueryExprs),
                             childPlan, context.getSubqueryToMarkJoinSlot(),
                             ctx.cascadesContext,
                             Optional.of(newProject), true, false);
@@ -240,9 +242,11 @@ public class SubqueryToApply implements AnalysisRuleFactory {
                         * if it's semi join with non-null mark slot
                         * we can safely change the mark conjunct to hash conjunct
                         */
+                        ExpressionRewriteContext rewriteContext = new ExpressionRewriteContext(ctx.cascadesContext);
                         boolean isMarkSlotNotNull = conjunct.containsType(MarkJoinSlotReference.class)
                                 ? ExpressionUtils.canInferNotNullForMarkSlot(
-                                    TrySimplifyPredicateWithMarkJoinSlot.INSTANCE.rewrite(conjunct, null))
+                                    TrySimplifyPredicateWithMarkJoinSlot.INSTANCE.rewrite(conjunct, rewriteContext),
+                                    rewriteContext)
                                 : false;
                         applyPlan = subqueryToApply(
                                 subqueryExprs.stream().collect(ImmutableList.toImmutableList()),
@@ -566,4 +570,33 @@ public class SubqueryToApply implements AnalysisRuleFactory {
         }
         return false;
     }
+
+    private List<Boolean> shouldOutputMarkJoinSlot(Collection<Expression> conjuncts) {
+        ImmutableList.Builder<Boolean> result = ImmutableList.builderWithExpectedSize(conjuncts.size());
+        for (Expression expr : conjuncts) {
+            result.add(!(expr instanceof SubqueryExpr) && expr.containsType(SubqueryExpr.class));
+        }
+        return result.build();
+    }
+
+    private CollectSubquerys collectSubquerys(Collection<? extends Expression> exprs) {
+        boolean hasSubqueryExpr = false;
+        ImmutableList.Builder<Set<SubqueryExpr>> subqueryExprsListBuilder = ImmutableList.builder();
+        for (Expression expression : exprs) {
+            Set<SubqueryExpr> subqueries = expression.collect(SubqueryToApply::canConvertToSupply);
+            hasSubqueryExpr |= !subqueries.isEmpty();
+            subqueryExprsListBuilder.add(subqueries);
+        }
+        return new CollectSubquerys(subqueryExprsListBuilder.build(), hasSubqueryExpr);
+    }
+
+    private static class CollectSubquerys {
+        final List<Set<SubqueryExpr>> subqueies;
+        final boolean hasSubquery;
+
+        public CollectSubquerys(List<Set<SubqueryExpr>> subqueies, boolean hasSubquery) {
+            this.subqueies = subqueies;
+            this.hasSubquery = hasSubquery;
+        }
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java
index 66852444cc0..90e0f8ed1db 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java
@@ -122,10 +122,11 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac
         // TODO Just Check query queryPlan firstly, support multi later.
         StructInfo queryStructInfo = queryStructInfos.get(0);
         if (!checkPattern(queryStructInfo)) {
-            cascadesContext.getMaterializationContexts().forEach(ctx ->
-                    ctx.recordFailReason(queryStructInfo, "Query struct info is invalid",
-                            () -> String.format("queryPlan is %s", queryPlan.treeString())
-                    ));
+            for (MaterializationContext ctx : cascadesContext.getMaterializationContexts()) {
+                ctx.recordFailReason(queryStructInfo, "Query struct info is invalid",
+                        () -> String.format("queryPlan is %s", queryPlan.treeString())
+                );
+            }
             return validQueryStructInfos;
         }
         validQueryStructInfos.add(queryStructInfo);
@@ -228,7 +229,7 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac
                                     viewToQuerySlotMapping));
                     continue;
                 }
-                rewrittenPlan = new LogicalFilter<>(Sets.newHashSet(rewriteCompensatePredicates), mvScan);
+                rewrittenPlan = new LogicalFilter<>(Sets.newLinkedHashSet(rewriteCompensatePredicates), mvScan);
             }
             // Rewrite query by view
             rewrittenPlan = rewriteQueryByView(matchMode, queryStructInfo, viewStructInfo, viewToQuerySlotMapping,
@@ -293,7 +294,7 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac
         if (originOutputs.size() != rewrittenPlan.getOutput().size()) {
             return null;
         }
-        Map<Slot, ExprId> originSlotToRewrittenExprId = Maps.newHashMap();
+        Map<Slot, ExprId> originSlotToRewrittenExprId = Maps.newLinkedHashMap();
         for (int i = 0; i < originOutputs.size(); i++) {
             originSlotToRewrittenExprId.put(originOutputs.get(i), rewrittenPlan.getOutput().get(i).getExprId());
         }
@@ -305,7 +306,7 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac
         rewrittenPlan = rewrittenPlanContext.getRewritePlan();
 
         // for get right nullable after rewritten, we need this map
-        Map<ExprId, Slot> exprIdToNewRewrittenSlot = Maps.newHashMap();
+        Map<ExprId, Slot> exprIdToNewRewrittenSlot = Maps.newLinkedHashMap();
         for (Slot slot : rewrittenPlan.getOutput()) {
             exprIdToNewRewrittenSlot.put(slot.getExprId(), slot);
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java
index 914b2133613..2d5c9bf377b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java
@@ -79,7 +79,7 @@ public class InitMaterializationContextHook implements PlannerHook {
         if (availableMTMVs.isEmpty()) {
             return;
         }
-        availableMTMVs.forEach(materializedView -> {
+        for (MTMV materializedView : availableMTMVs) {
             // generate outside, maybe add partition filter in the future
             LogicalOlapScan mvScan = new LogicalOlapScan(
                     cascadesContext.getStatementContext().getNextRelationId(),
@@ -96,6 +96,6 @@ public class InitMaterializationContextHook implements PlannerHook {
             Plan projectScan = new LogicalProject<Plan>(mvProjects, mvScan);
             cascadesContext.addMaterializationContext(
                     MaterializationContext.fromMaterializedView(materializedView, projectScan, cascadesContext));
-        });
+        }
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java
index b3de7443652..db9f58ae070 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java
@@ -33,11 +33,12 @@ import org.apache.doris.nereids.util.ExpressionUtils;
 import org.apache.doris.nereids.util.Utils;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -67,7 +68,7 @@ public class MaterializationContext {
     private boolean success = false;
     // if rewrite by mv fail, record the reason, if success the failReason should be empty.
     // The key is the query belonged group expression objectId, the value is the fail reason
-    private final Map<ObjectId, Pair<String, String>> failReason = new HashMap<>();
+    private final Map<ObjectId, Pair<String, String>> failReason = new LinkedHashMap<>();
     private boolean enableRecordFailureDetail = false;
 
     /**
@@ -163,7 +164,6 @@ public class MaterializationContext {
         if (this.success) {
             return;
         }
-        this.success = false;
         this.failReason.put(structInfo.getOriginalPlanId(),
                 Pair.of(summary, this.isEnableRecordFailureDetail() ? failureReasonSupplier.get() : ""));
     }
@@ -233,7 +233,7 @@ public class MaterializationContext {
         for (MaterializationContext ctx : materializationContexts) {
             if (!ctx.isSuccess()) {
                 Set<String> failReasonSet =
-                        ctx.getFailReason().values().stream().map(Pair::key).collect(Collectors.toSet());
+                        ctx.getFailReason().values().stream().map(Pair::key).collect(ImmutableSet.toImmutableSet());
                 builder.append("\n")
                         .append("  Name: ").append(ctx.getMTMV().getName())
                         .append("\n")
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java
index 674d4935594..0d280bb8340 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java
@@ -53,8 +53,8 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -116,7 +116,7 @@ public class StructInfo {
         this.predicates = predicates;
         if (predicates == null) {
             // collect predicate from top plan which not in hyper graph
-            Set<Expression> topPlanPredicates = new HashSet<>();
+            Set<Expression> topPlanPredicates = new LinkedHashSet<>();
             topPlan.accept(PREDICATE_COLLECTOR, topPlanPredicates);
             this.predicates = Predicates.of(topPlanPredicates);
         }
@@ -241,7 +241,9 @@ public class StructInfo {
     public static List<StructInfo> of(Plan originalPlan) {
         // TODO only consider the inner join currently, Should support outer join
         // Split plan by the boundary which contains multi child
-        PlanSplitContext planSplitContext = new PlanSplitContext(Sets.newHashSet(LogicalJoin.class));
+        LinkedHashSet<Class<? extends Plan>> set = Sets.newLinkedHashSet();
+        set.add(LogicalJoin.class);
+        PlanSplitContext planSplitContext = new PlanSplitContext(set);
         // if single table without join, the bottom is
         originalPlan.accept(PLAN_SPLITTER, planSplitContext);
 
@@ -261,16 +263,18 @@ public class StructInfo {
                 .map(GroupExpression::getId).orElseGet(() -> new ObjectId(-1));
         // if any of topPlan or bottomPlan is null, split the top plan to two parts by join node
         if (topPlan == null || bottomPlan == null) {
-            PlanSplitContext planSplitContext = new PlanSplitContext(Sets.newHashSet(LogicalJoin.class));
+            Set<Class<? extends Plan>> set = Sets.newLinkedHashSet();
+            set.add(LogicalJoin.class);
+            PlanSplitContext planSplitContext = new PlanSplitContext(set);
             originalPlan.accept(PLAN_SPLITTER, planSplitContext);
             bottomPlan = planSplitContext.getBottomPlan();
             topPlan = planSplitContext.getTopPlan();
         }
         // collect struct info fromGraph
         ImmutableList.Builder<CatalogRelation> relationBuilder = ImmutableList.builder();
-        Map<RelationId, StructInfoNode> relationIdStructInfoNodeMap = new HashMap<>();
-        Map<Expression, Expression> shuttledHashConjunctsToConjunctsMap = new HashMap<>();
-        Map<ExprId, Expression> namedExprIdAndExprMapping = new HashMap<>();
+        Map<RelationId, StructInfoNode> relationIdStructInfoNodeMap = new LinkedHashMap<>();
+        Map<Expression, Expression> shuttledHashConjunctsToConjunctsMap = new LinkedHashMap<>();
+        Map<ExprId, Expression> namedExprIdAndExprMapping = new LinkedHashMap<>();
         boolean valid = collectStructInfoFromGraph(hyperGraph, topPlan, shuttledHashConjunctsToConjunctsMap,
                 namedExprIdAndExprMapping,
                 relationBuilder,
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionBottomUpRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionBottomUpRewriter.java
new file mode 100644
index 00000000000..932446ce48b
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionBottomUpRewriter.java
@@ -0,0 +1,124 @@
+// 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.expression;
+
+import org.apache.doris.nereids.pattern.ExpressionPatternRules;
+import org.apache.doris.nereids.pattern.ExpressionPatternTraverseListeners;
+import org.apache.doris.nereids.pattern.ExpressionPatternTraverseListeners.CombinedListener;
+import org.apache.doris.nereids.trees.expressions.Expression;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.annotation.Nullable;
+
+/** ExpressionBottomUpRewriter */
+public class ExpressionBottomUpRewriter implements ExpressionRewriteRule<ExpressionRewriteContext> {
+    public static final String BATCH_ID_KEY = "batch_id";
+    private static final Logger LOG = LogManager.getLogger(ExpressionBottomUpRewriter.class);
+    private static final AtomicInteger rewriteBatchId = new AtomicInteger();
+    private final ExpressionPatternRules rules;
+    private final ExpressionPatternTraverseListeners listeners;
+
+    public ExpressionBottomUpRewriter(ExpressionPatternRules rules, ExpressionPatternTraverseListeners listeners) {
+        this.rules = rules;
+        this.listeners = listeners;
+    }
+
+    // entrance
+    @Override
+    public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) {
+        int currentBatch = rewriteBatchId.incrementAndGet();
+        return rewriteBottomUp(expr, ctx, currentBatch, null, rules, listeners);
+    }
+
+    private static Expression rewriteBottomUp(
+            Expression expression, ExpressionRewriteContext context, int currentBatch, @Nullable Expression parent,
+            ExpressionPatternRules rules, ExpressionPatternTraverseListeners listeners) {
+
+        Optional<Integer> rewriteState = expression.getMutableState(BATCH_ID_KEY);
+        if (!rewriteState.isPresent() || rewriteState.get() != currentBatch) {
+            CombinedListener listener = null;
+            boolean hasChildren = expression.arity() > 0;
+            if (hasChildren) {
+                listener = listeners.matchesAndCombineListeners(expression, context, parent);
+                if (listener != null) {
+                    listener.onEnter();
+                }
+            }
+
+            Expression afterRewrite = expression;
+            try {
+                Expression beforeRewrite;
+                afterRewrite = rewriteChildren(expression, context, currentBatch, rules, listeners);
+                // use rewriteTimes to avoid dead loop
+                int rewriteTimes = 0;
+                boolean changed;
+                do {
+                    beforeRewrite = afterRewrite;
+
+                    // rewrite this
+                    Optional<Expression> applied = rules.matchesAndApply(beforeRewrite, context, parent);
+
+                    changed = applied.isPresent();
+                    if (changed) {
+                        afterRewrite = applied.get();
+                        // ensure children are rewritten
+                        afterRewrite = rewriteChildren(afterRewrite, context, currentBatch, rules, listeners);
+                    }
+                    rewriteTimes++;
+                } while (changed && rewriteTimes < 100);
+
+                // set rewritten
+                afterRewrite.setMutableState(BATCH_ID_KEY, currentBatch);
+            } finally {
+                if (hasChildren && listener != null) {
+                    listener.onExit(afterRewrite);
+                }
+            }
+
+            return afterRewrite;
+        }
+
+        // already rewritten
+        return expression;
+    }
+
+    private static Expression rewriteChildren(Expression parent, ExpressionRewriteContext context, int currentBatch,
+            ExpressionPatternRules rules, ExpressionPatternTraverseListeners listeners) {
+        boolean changed = false;
+        ImmutableList.Builder<Expression> newChildren = ImmutableList.builderWithExpectedSize(parent.arity());
+        for (Expression child : parent.children()) {
+            Expression newChild = rewriteBottomUp(child, context, currentBatch, parent, rules, listeners);
+            changed |= !child.equals(newChild);
+            newChildren.add(newChild);
+        }
+
+        Expression result = parent;
+        if (changed) {
+            result = parent.withChildren(newChildren.build());
+        }
+        if (changed && context.cascadesContext.isEnableExprTrace()) {
+            LOG.info("WithChildren: \nbefore: " + parent + "\nafter: " + result);
+        }
+        return result;
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionListenerMatcher.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionListenerMatcher.java
new file mode 100644
index 00000000000..ea67d14e8fe
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionListenerMatcher.java
@@ -0,0 +1,41 @@
+// 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.expression;
+
+import org.apache.doris.nereids.trees.expressions.Expression;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Predicate;
+
+/** ExpressionListenerMatcher */
+public class ExpressionListenerMatcher<E extends Expression> {
+    public final Class<E> typePattern;
+    public final List<Predicate<ExpressionMatchingContext<E>>> predicates;
+    public final ExpressionTraverseListener<E> listener;
+
+    public ExpressionListenerMatcher(Class<E> typePattern,
+            List<Predicate<ExpressionMatchingContext<E>>> predicates,
+            ExpressionTraverseListener<E> listener) {
+        this.typePattern = Objects.requireNonNull(typePattern, "typePattern can not be null");
+        this.predicates = predicates == null ? ImmutableList.of() : predicates;
+        this.listener = Objects.requireNonNull(listener, "listener can not be null");
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionMatchingAction.java
similarity index 73%
copy from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
copy to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionMatchingAction.java
index cb50e0d2871..a28b96079b5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionMatchingAction.java
@@ -17,17 +17,9 @@
 
 package org.apache.doris.nereids.rules.expression;
 
-import org.apache.doris.nereids.CascadesContext;
-
-/**
- * expression rewrite context.
- */
-public class ExpressionRewriteContext {
-
-    public final CascadesContext cascadesContext;
-
-    public ExpressionRewriteContext(CascadesContext cascadesContext) {
-        this.cascadesContext = cascadesContext;
-    }
+import org.apache.doris.nereids.trees.expressions.Expression;
 
+/** ExpressionMatchAction */
+public interface ExpressionMatchingAction<E extends Expression> {
+    Expression apply(ExpressionMatchingContext<E> context);
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionMatchingContext.java
similarity index 55%
copy from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
copy to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionMatchingContext.java
index cb50e0d2871..953815ad87c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionMatchingContext.java
@@ -18,16 +18,29 @@
 package org.apache.doris.nereids.rules.expression;
 
 import org.apache.doris.nereids.CascadesContext;
+import org.apache.doris.nereids.trees.expressions.Expression;
 
-/**
- * expression rewrite context.
- */
-public class ExpressionRewriteContext {
+import java.util.Optional;
 
+/** ExpressionMatchingContext */
+public class ExpressionMatchingContext<E extends Expression> {
+    public final E expr;
+    public final Optional<Expression> parent;
+    public final ExpressionRewriteContext rewriteContext;
     public final CascadesContext cascadesContext;
 
-    public ExpressionRewriteContext(CascadesContext cascadesContext) {
-        this.cascadesContext = cascadesContext;
+    public ExpressionMatchingContext(E expr, Expression parent, ExpressionRewriteContext context) {
+        this.expr = expr;
+        this.parent = Optional.ofNullable(parent);
+        this.rewriteContext = context;
+        this.cascadesContext = context.cascadesContext;
     }
 
+    public boolean isRoot() {
+        return !parent.isPresent();
+    }
+
+    public Expression parentOr(Expression defaultParent) {
+        return parent.orElse(defaultParent);
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalization.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalization.java
index 9886cb1787e..adf0cb90a95 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalization.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalization.java
@@ -42,20 +42,21 @@ public class ExpressionNormalization extends ExpressionRewrite {
     // we should run supportJavaDateFormatter before foldConstantRule or be will fold
     // from_unixtime(timestamp, 'yyyyMMdd') to 'yyyyMMdd'
     public static final List<ExpressionRewriteRule> NORMALIZE_REWRITE_RULES = ImmutableList.of(
-            SupportJavaDateFormatter.INSTANCE,
-            ReplaceVariableByLiteral.INSTANCE,
-            NormalizeBinaryPredicatesRule.INSTANCE,
-            InPredicateDedup.INSTANCE,
-            InPredicateToEqualToRule.INSTANCE,
-            SimplifyNotExprRule.INSTANCE,
-            SimplifyArithmeticRule.INSTANCE,
-            FoldConstantRule.INSTANCE,
-            SimplifyCastRule.INSTANCE,
-            DigitalMaskingConvert.INSTANCE,
-            SimplifyArithmeticComparisonRule.INSTANCE,
-            SupportJavaDateFormatter.INSTANCE,
-            ConvertAggStateCast.INSTANCE,
-            CheckCast.INSTANCE
+            bottomUp(
+                ReplaceVariableByLiteral.INSTANCE,
+                SupportJavaDateFormatter.INSTANCE,
+                NormalizeBinaryPredicatesRule.INSTANCE,
+                InPredicateDedup.INSTANCE,
+                InPredicateToEqualToRule.INSTANCE,
+                SimplifyNotExprRule.INSTANCE,
+                SimplifyArithmeticRule.INSTANCE,
+                FoldConstantRule.INSTANCE,
+                SimplifyCastRule.INSTANCE,
+                DigitalMaskingConvert.INSTANCE,
+                SimplifyArithmeticComparisonRule.INSTANCE,
+                ConvertAggStateCast.INSTANCE,
+                CheckCast.INSTANCE
+            )
     );
 
     public ExpressionNormalization() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalizationAndOptimization.java
similarity index 59%
copy from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
copy to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalizationAndOptimization.java
index cb50e0d2871..d694062ef1f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionNormalizationAndOptimization.java
@@ -17,17 +17,17 @@
 
 package org.apache.doris.nereids.rules.expression;
 
-import org.apache.doris.nereids.CascadesContext;
+import com.google.common.collect.ImmutableList;
 
-/**
- * expression rewrite context.
- */
-public class ExpressionRewriteContext {
-
-    public final CascadesContext cascadesContext;
-
-    public ExpressionRewriteContext(CascadesContext cascadesContext) {
-        this.cascadesContext = cascadesContext;
+/** ExpressionNormalizationAndOptimization */
+public class ExpressionNormalizationAndOptimization extends ExpressionRewrite {
+    /** ExpressionNormalizationAndOptimization */
+    public ExpressionNormalizationAndOptimization() {
+        super(new ExpressionRuleExecutor(
+                ImmutableList.<ExpressionRewriteRule>builder()
+                        .addAll(ExpressionNormalization.NORMALIZE_REWRITE_RULES)
+                        .addAll(ExpressionOptimization.OPTIMIZE_REWRITE_RULES)
+                        .build()
+        ));
     }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java
index fdf9820c582..b3bb18163ea 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java
@@ -39,18 +39,20 @@ import java.util.List;
  */
 public class ExpressionOptimization extends ExpressionRewrite {
     public static final List<ExpressionRewriteRule> OPTIMIZE_REWRITE_RULES = ImmutableList.of(
-            ExtractCommonFactorRule.INSTANCE,
-            DistinctPredicatesRule.INSTANCE,
-            SimplifyComparisonPredicate.INSTANCE,
-            SimplifyInPredicate.INSTANCE,
-            SimplifyDecimalV3Comparison.INSTANCE,
-            SimplifyRange.INSTANCE,
-            DateFunctionRewrite.INSTANCE,
-            OrToIn.INSTANCE,
-            ArrayContainToArrayOverlap.INSTANCE,
-            CaseWhenToIf.INSTANCE,
-            TopnToMax.INSTANCE,
-            NullSafeEqualToEqual.INSTANCE
+            bottomUp(
+                ExtractCommonFactorRule.INSTANCE,
+                DistinctPredicatesRule.INSTANCE,
+                SimplifyComparisonPredicate.INSTANCE,
+                SimplifyInPredicate.INSTANCE,
+                SimplifyDecimalV3Comparison.INSTANCE,
+                OrToIn.INSTANCE,
+                SimplifyRange.INSTANCE,
+                DateFunctionRewrite.INSTANCE,
+                ArrayContainToArrayOverlap.INSTANCE,
+                CaseWhenToIf.INSTANCE,
+                TopnToMax.INSTANCE,
+                NullSafeEqualToEqual.INSTANCE
+            )
     );
     private static final ExpressionRuleExecutor EXECUTOR = new ExpressionRuleExecutor(OPTIMIZE_REWRITE_RULES);
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternMatchRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternMatchRule.java
new file mode 100644
index 00000000000..dbf5c79c96d
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternMatchRule.java
@@ -0,0 +1,64 @@
+// 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.expression;
+
+import org.apache.doris.nereids.pattern.TypeMappings.TypeMapping;
+import org.apache.doris.nereids.trees.expressions.Expression;
+
+import java.util.List;
+import java.util.function.Predicate;
+
+/** ExpressionPatternMatcherRule */
+public class ExpressionPatternMatchRule implements TypeMapping<Expression> {
+    public final Class<? extends Expression> typePattern;
+    public final List<Predicate<ExpressionMatchingContext<Expression>>> predicates;
+    public final ExpressionMatchingAction<Expression> matchingAction;
+
+    public ExpressionPatternMatchRule(ExpressionPatternMatcher patternMatcher) {
+        this.typePattern = patternMatcher.typePattern;
+        this.predicates = patternMatcher.predicates;
+        this.matchingAction = patternMatcher.matchingAction;
+    }
+
+    /** matches */
+    public boolean matchesTypeAndPredicates(ExpressionMatchingContext<Expression> context) {
+        return typePattern.isInstance(context.expr) && matchesPredicates(context);
+    }
+
+    /** matchesPredicates */
+    public boolean matchesPredicates(ExpressionMatchingContext<Expression> context) {
+        if (!predicates.isEmpty()) {
+            for (Predicate<ExpressionMatchingContext<Expression>> predicate : predicates) {
+                if (!predicate.test(context)) {
+                    return false;
+                }
+            }
+        }
+        return true;
+    }
+
+    public Expression apply(ExpressionMatchingContext<Expression> context) {
+        Expression newResult = matchingAction.apply(context);
+        return newResult == null ? context.expr : newResult;
+    }
+
+    @Override
+    public Class<? extends Expression> getType() {
+        return typePattern;
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternMatcher.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternMatcher.java
new file mode 100644
index 00000000000..058b1d60b1d
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternMatcher.java
@@ -0,0 +1,41 @@
+// 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.expression;
+
+import org.apache.doris.nereids.trees.expressions.Expression;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Predicate;
+
+/** ExpressionPattern */
+public class ExpressionPatternMatcher<E extends Expression> {
+    public final Class<E> typePattern;
+    public final List<Predicate<ExpressionMatchingContext<E>>> predicates;
+    public final ExpressionMatchingAction<E> matchingAction;
+
+    public ExpressionPatternMatcher(Class<E> typePattern,
+            List<Predicate<ExpressionMatchingContext<E>>> predicates,
+            ExpressionMatchingAction<E> matchingAction) {
+        this.typePattern = Objects.requireNonNull(typePattern, "typePattern can not be null");
+        this.predicates = predicates == null ? ImmutableList.of() : predicates;
+        this.matchingAction = Objects.requireNonNull(matchingAction, "matchingAction can not be null");
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternRuleFactory.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternRuleFactory.java
new file mode 100644
index 00000000000..7fb18735ba5
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionPatternRuleFactory.java
@@ -0,0 +1,84 @@
+// 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.expression;
+
+import org.apache.doris.nereids.trees.expressions.Expression;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/** ExpressionPatternRuleFactory */
+public interface ExpressionPatternRuleFactory {
+    List<ExpressionPatternMatcher<? extends Expression>> buildRules();
+
+    default <E extends Expression> ExpressionPatternDescriptor<E> matchesType(Class<E> clazz) {
+        return new ExpressionPatternDescriptor<>(clazz);
+    }
+
+    default <E extends Expression> ExpressionPatternDescriptor<E> root(Class<E> clazz) {
+        return new ExpressionPatternDescriptor<>(clazz)
+                .whenCtx(ctx -> ctx.isRoot());
+    }
+
+    default <E extends Expression> ExpressionPatternDescriptor<E> matchesTopType(Class<E> clazz) {
+        return new ExpressionPatternDescriptor<>(clazz)
+                .whenCtx(ctx -> ctx.isRoot() || !clazz.isInstance(ctx.parent.get()));
+    }
+
+    /** ExpressionPatternDescriptor */
+    class ExpressionPatternDescriptor<E extends Expression> {
+        private final Class<E> typePattern;
+        private final ImmutableList<Predicate<ExpressionMatchingContext<E>>> predicates;
+
+        public ExpressionPatternDescriptor(Class<E> typePattern) {
+            this(typePattern, ImmutableList.of());
+        }
+
+        public ExpressionPatternDescriptor(
+                Class<E> typePattern, ImmutableList<Predicate<ExpressionMatchingContext<E>>> predicates) {
+            this.typePattern = Objects.requireNonNull(typePattern, "typePattern can not be null");
+            this.predicates = Objects.requireNonNull(predicates, "predicates can not be null");
+        }
+
+        public ExpressionPatternDescriptor<E> when(Predicate<E> predicate) {
+            return whenCtx(ctx -> predicate.test(ctx.expr));
+        }
+
+        public ExpressionPatternDescriptor<E> whenCtx(Predicate<ExpressionMatchingContext<E>> predicate) {
+            ImmutableList.Builder<Predicate<ExpressionMatchingContext<E>>> newPredicates
+                    = ImmutableList.builderWithExpectedSize(predicates.size() + 1);
+            newPredicates.addAll(predicates);
+            newPredicates.add(predicate);
+            return new ExpressionPatternDescriptor<>(typePattern, newPredicates.build());
+        }
+
+        /** then */
+        public ExpressionPatternMatcher<E> then(Function<E, Expression> rewriter) {
+            return new ExpressionPatternMatcher<>(
+                    typePattern, predicates, (context) -> rewriter.apply(context.expr));
+        }
+
+        public ExpressionPatternMatcher<E> thenApply(ExpressionMatchingAction<E> action) {
+            return new ExpressionPatternMatcher<>(typePattern, predicates, action);
+        }
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java
index 912793e61d1..b547f693a7c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java
@@ -18,6 +18,8 @@
 package org.apache.doris.nereids.rules.expression;
 
 import org.apache.doris.common.Pair;
+import org.apache.doris.nereids.pattern.ExpressionPatternRules;
+import org.apache.doris.nereids.pattern.ExpressionPatternTraverseListeners;
 import org.apache.doris.nereids.properties.OrderKey;
 import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.rules.RuleType;
@@ -41,7 +43,7 @@ import org.apache.doris.nereids.util.ExpressionUtils;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
@@ -123,9 +125,7 @@ public class ExpressionRewrite implements RewriteRuleFactory {
                 LogicalProject<Plan> project = ctx.root;
                 ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext);
                 List<NamedExpression> projects = project.getProjects();
-                List<NamedExpression> newProjects = projects.stream()
-                        .map(expr -> (NamedExpression) rewriter.rewrite(expr, context))
-                        .collect(ImmutableList.toImmutableList());
+                List<NamedExpression> newProjects = rewriteAll(projects, rewriter, context);
                 if (projects.equals(newProjects)) {
                     return project;
                 }
@@ -160,9 +160,7 @@ public class ExpressionRewrite implements RewriteRuleFactory {
                 List<Expression> newGroupByExprs = rewriter.rewrite(groupByExprs, context);
 
                 List<NamedExpression> outputExpressions = agg.getOutputExpressions();
-                List<NamedExpression> newOutputExpressions = outputExpressions.stream()
-                        .map(expr -> (NamedExpression) rewriter.rewrite(expr, context))
-                        .collect(ImmutableList.toImmutableList());
+                List<NamedExpression> newOutputExpressions = rewriteAll(outputExpressions, rewriter, context);
                 if (outputExpressions.equals(newOutputExpressions)) {
                     return agg;
                 }
@@ -222,13 +220,16 @@ public class ExpressionRewrite implements RewriteRuleFactory {
             return logicalSort().thenApply(ctx -> {
                 LogicalSort<Plan> sort = ctx.root;
                 List<OrderKey> orderKeys = sort.getOrderKeys();
-                List<OrderKey> rewrittenOrderKeys = new ArrayList<>();
+                ImmutableList.Builder<OrderKey> rewrittenOrderKeys
+                        = ImmutableList.builderWithExpectedSize(orderKeys.size());
                 ExpressionRewriteContext context = new ExpressionRewriteContext(ctx.cascadesContext);
+                boolean changed = false;
                 for (OrderKey k : orderKeys) {
                     Expression expression = rewriter.rewrite(k.getExpr(), context);
+                    changed |= expression != k.getExpr();
                     rewrittenOrderKeys.add(new OrderKey(expression, k.isAsc(), k.isNullFirst()));
                 }
-                return sort.withOrderKeys(rewrittenOrderKeys);
+                return changed ? sort.withOrderKeys(rewrittenOrderKeys.build()) : sort;
             }).toRule(RuleType.REWRITE_SORT_EXPRESSION);
         }
     }
@@ -270,4 +271,36 @@ public class ExpressionRewrite implements RewriteRuleFactory {
             }).toRule(RuleType.REWRITE_REPEAT_EXPRESSION);
         }
     }
+
+    /** bottomUp */
+    public static ExpressionBottomUpRewriter bottomUp(ExpressionPatternRuleFactory... ruleFactories) {
+        ImmutableList.Builder<ExpressionPatternMatchRule> rules = ImmutableList.builder();
+        ImmutableList.Builder<ExpressionTraverseListenerMapping> listeners = ImmutableList.builder();
+        for (ExpressionPatternRuleFactory ruleFactory : ruleFactories) {
+            if (ruleFactory instanceof ExpressionTraverseListenerFactory) {
+                List<ExpressionListenerMatcher<? extends Expression>> listenersMatcher
+                        = ((ExpressionTraverseListenerFactory) ruleFactory).buildListeners();
+                for (ExpressionListenerMatcher<? extends Expression> listenerMatcher : listenersMatcher) {
+                    listeners.add(new ExpressionTraverseListenerMapping(listenerMatcher));
+                }
+            }
+            for (ExpressionPatternMatcher<? extends Expression> patternMatcher : ruleFactory.buildRules()) {
+                rules.add(new ExpressionPatternMatchRule(patternMatcher));
+            }
+        }
+
+        return new ExpressionBottomUpRewriter(
+                new ExpressionPatternRules(rules.build()),
+                new ExpressionPatternTraverseListeners(listeners.build())
+        );
+    }
+
+    public static <E extends Expression> List<E> rewriteAll(
+            Collection<E> exprs, ExpressionRuleExecutor rewriter, ExpressionRewriteContext context) {
+        ImmutableList.Builder<E> result = ImmutableList.builderWithExpectedSize(exprs.size());
+        for (E expr : exprs) {
+            result.add((E) rewriter.rewrite(expr, context));
+        }
+        return result.build();
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
index cb50e0d2871..35633e7594f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
@@ -19,6 +19,8 @@ package org.apache.doris.nereids.rules.expression;
 
 import org.apache.doris.nereids.CascadesContext;
 
+import java.util.Objects;
+
 /**
  * expression rewrite context.
  */
@@ -27,7 +29,7 @@ public class ExpressionRewriteContext {
     public final CascadesContext cascadesContext;
 
     public ExpressionRewriteContext(CascadesContext cascadesContext) {
-        this.cascadesContext = cascadesContext;
+        this.cascadesContext = Objects.requireNonNull(cascadesContext, "cascadesContext can not be null");
     }
 
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleExecutor.java
index ac7e6dae6b2..0f951448dd2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleExecutor.java
@@ -18,6 +18,7 @@
 package org.apache.doris.nereids.rules.expression;
 
 import org.apache.doris.nereids.rules.expression.rules.NormalizeBinaryPredicatesRule;
+import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
 import org.apache.doris.nereids.trees.expressions.Expression;
 
 import com.google.common.collect.ImmutableList;
@@ -36,7 +37,11 @@ public class ExpressionRuleExecutor {
     }
 
     public List<Expression> rewrite(List<Expression> exprs, ExpressionRewriteContext ctx) {
-        return exprs.stream().map(expr -> rewrite(expr, ctx)).collect(ImmutableList.toImmutableList());
+        ImmutableList.Builder<Expression> result = ImmutableList.builderWithExpectedSize(exprs.size());
+        for (Expression expr : exprs) {
+            result.add(rewrite(expr, ctx));
+        }
+        return result.build();
     }
 
     /**
@@ -61,8 +66,15 @@ public class ExpressionRuleExecutor {
         return rule.rewrite(expr, ctx);
     }
 
+    /** normalize */
     public static Expression normalize(Expression expression) {
-        return NormalizeBinaryPredicatesRule.INSTANCE.rewrite(expression, null);
+        return expression.rewriteUp(expr -> {
+            if (expr instanceof ComparisonPredicate) {
+                return NormalizeBinaryPredicatesRule.normalize((ComparisonPredicate) expression);
+            } else {
+                return expr;
+            }
+        });
     }
 
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListener.java
similarity index 66%
copy from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
copy to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListener.java
index cb50e0d2871..5df5a6d6818 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListener.java
@@ -17,17 +17,15 @@
 
 package org.apache.doris.nereids.rules.expression;
 
-import org.apache.doris.nereids.CascadesContext;
+import org.apache.doris.nereids.trees.expressions.Expression;
 
-/**
- * expression rewrite context.
- */
-public class ExpressionRewriteContext {
+/** ExpressionTraverseListener */
+public interface ExpressionTraverseListener<E extends Expression> {
+    default void onEnter(ExpressionMatchingContext<E> context) {}
 
-    public final CascadesContext cascadesContext;
+    default void onExit(ExpressionMatchingContext<E> context, Expression rewritten) {}
 
-    public ExpressionRewriteContext(CascadesContext cascadesContext) {
-        this.cascadesContext = cascadesContext;
+    default <CAST extends Expression> ExpressionTraverseListener<CAST> as() {
+        return (ExpressionTraverseListener) this;
     }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListenerFactory.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListenerFactory.java
new file mode 100644
index 00000000000..201362fed78
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListenerFactory.java
@@ -0,0 +1,79 @@
+// 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.expression;
+
+import org.apache.doris.nereids.trees.expressions.Expression;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Predicate;
+
+/** ExpressionTraverseListenerFactory */
+public interface ExpressionTraverseListenerFactory {
+    List<ExpressionListenerMatcher<? extends Expression>> buildListeners();
+
+    default <E extends Expression> ListenerDescriptor<E> listenerType(Class<E> clazz) {
+        return new ListenerDescriptor<>(clazz);
+    }
+
+    /** listenerTypes */
+    default List<ListenerDescriptor<Expression>> listenerTypes(Class<? extends Expression>... classes) {
+        ImmutableList.Builder<ListenerDescriptor<Expression>> listeners
+                = ImmutableList.builderWithExpectedSize(classes.length);
+        for (Class<? extends Expression> clazz : classes) {
+            listeners.add((ListenerDescriptor<Expression>) listenerType(clazz));
+        }
+        return listeners.build();
+    }
+
+    /** ListenerDescriptor */
+    class ListenerDescriptor<E extends Expression> {
+
+        private final Class<E> typePattern;
+        private final ImmutableList<Predicate<ExpressionMatchingContext<E>>> predicates;
+
+        public ListenerDescriptor(Class<E> typePattern) {
+            this(typePattern, ImmutableList.of());
+        }
+
+        public ListenerDescriptor(
+                Class<E> typePattern, ImmutableList<Predicate<ExpressionMatchingContext<E>>> predicates) {
+            this.typePattern = Objects.requireNonNull(typePattern, "typePattern can not be null");
+            this.predicates = Objects.requireNonNull(predicates, "predicates can not be null");
+        }
+
+        public ListenerDescriptor<E> when(Predicate<E> predicate) {
+            return whenCtx(ctx -> predicate.test(ctx.expr));
+        }
+
+        public ListenerDescriptor<E> whenCtx(Predicate<ExpressionMatchingContext<E>> predicate) {
+            ImmutableList.Builder<Predicate<ExpressionMatchingContext<E>>> newPredicates
+                    = ImmutableList.builderWithExpectedSize(predicates.size() + 1);
+            newPredicates.addAll(predicates);
+            newPredicates.add(predicate);
+            return new ListenerDescriptor<>(typePattern, newPredicates.build());
+        }
+
+        /** then */
+        public ExpressionListenerMatcher<E> then(ExpressionTraverseListener<E> listener) {
+            return new ExpressionListenerMatcher<>(typePattern, predicates, listener);
+        }
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListenerMapping.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListenerMapping.java
new file mode 100644
index 00000000000..d99c231110f
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionTraverseListenerMapping.java
@@ -0,0 +1,59 @@
+// 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.expression;
+
+import org.apache.doris.nereids.pattern.TypeMappings.TypeMapping;
+import org.apache.doris.nereids.trees.expressions.Expression;
+
+import java.util.List;
+import java.util.function.Predicate;
+
+/** ExpressionTraverseListener */
+public class ExpressionTraverseListenerMapping implements TypeMapping<Expression> {
+    public final Class<? extends Expression> typePattern;
+    public final List<Predicate<ExpressionMatchingContext<Expression>>> predicates;
+    public final ExpressionTraverseListener<Expression> listener;
+
+    public ExpressionTraverseListenerMapping(ExpressionListenerMatcher listenerMatcher) {
+        this.typePattern = listenerMatcher.typePattern;
+        this.predicates = listenerMatcher.predicates;
+        this.listener = listenerMatcher.listener;
+    }
+
+    @Override
+    public Class<? extends Expression> getType() {
+        return typePattern;
+    }
+
+    /** matches */
+    public boolean matchesTypeAndPredicates(ExpressionMatchingContext<Expression> context) {
+        return typePattern.isInstance(context.expr) && matchesPredicates(context);
+    }
+
+    /** matchesPredicates */
+    public boolean matchesPredicates(ExpressionMatchingContext<Expression> context) {
+        if (!predicates.isEmpty()) {
+            for (Predicate<ExpressionMatchingContext<Expression>> predicate : predicates) {
+                if (!predicate.test(context)) {
+                    return false;
+                }
+            }
+        }
+        return true;
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java
index d7a6085dcab..69a9105d653 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/check/CheckCast.java
@@ -18,8 +18,8 @@
 package org.apache.doris.nereids.rules.expression.check;
 
 import org.apache.doris.nereids.exceptions.AnalysisException;
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.types.ArrayType;
@@ -31,18 +31,24 @@ import org.apache.doris.nereids.types.StructType;
 import org.apache.doris.nereids.types.coercion.CharacterType;
 import org.apache.doris.nereids.types.coercion.PrimitiveType;
 
+import com.google.common.collect.ImmutableList;
+
 import java.util.List;
 
 /**
  * check cast valid
  */
-public class CheckCast extends AbstractExpressionRewriteRule {
-
-    public static final CheckCast INSTANCE = new CheckCast();
+public class CheckCast implements ExpressionPatternRuleFactory {
+    public static CheckCast INSTANCE = new CheckCast();
 
     @Override
-    public Expression visitCast(Cast cast, ExpressionRewriteContext context) {
-        rewrite(cast.child(), context);
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(Cast.class).then(CheckCast::check)
+        );
+    }
+
+    private static Expression check(Cast cast) {
         DataType originalType = cast.child().getDataType();
         DataType targetType = cast.getDataType();
         if (!check(originalType, targetType)) {
@@ -51,7 +57,7 @@ public class CheckCast extends AbstractExpressionRewriteRule {
         return cast;
     }
 
-    private boolean check(DataType originalType, DataType targetType) {
+    private static boolean check(DataType originalType, DataType targetType) {
         if (originalType.isVariantType() && (targetType instanceof PrimitiveType || targetType.isArrayType())) {
             // variant could cast to primitive types and array
             return true;
@@ -99,7 +105,7 @@ public class CheckCast extends AbstractExpressionRewriteRule {
      *   3. original type is same with target type
      *   4. target type is null type
      */
-    private boolean checkPrimitiveType(DataType originalType, DataType targetType) {
+    private static boolean checkPrimitiveType(DataType originalType, DataType targetType) {
         if (!originalType.isPrimitive() || !targetType.isPrimitive()) {
             return false;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ArrayContainToArrayOverlap.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ArrayContainToArrayOverlap.java
index 7309ef111c9..f32d76062aa 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ArrayContainToArrayOverlap.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ArrayContainToArrayOverlap.java
@@ -17,26 +17,29 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteRule;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.Or;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.ArrayContains;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraysOverlap;
 import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
-import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter;
 import org.apache.doris.nereids.util.ExpressionUtils;
+import org.apache.doris.nereids.util.Utils;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList.Builder;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
 
-import java.util.HashMap;
-import java.util.HashSet;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
-import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * array_contains ( c_array, '1' )
@@ -44,56 +47,73 @@ import java.util.stream.Collectors;
  * =========================================>
  * array_overlap(c_array, ['1', '2'])
  */
-public class ArrayContainToArrayOverlap extends DefaultExpressionRewriter<ExpressionRewriteContext> implements
-        ExpressionRewriteRule<ExpressionRewriteContext> {
+public class ArrayContainToArrayOverlap implements ExpressionPatternRuleFactory {
 
     public static final ArrayContainToArrayOverlap INSTANCE = new ArrayContainToArrayOverlap();
 
     private static final int REWRITE_PREDICATE_THRESHOLD = 2;
 
     @Override
-    public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) {
-        return expr.accept(this, ctx);
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesTopType(Or.class).then(ArrayContainToArrayOverlap::rewrite)
+        );
     }
 
-    @Override
-    public Expression visitOr(Or or, ExpressionRewriteContext ctx) {
+    private static Expression rewrite(Or or) {
         List<Expression> disjuncts = ExpressionUtils.extractDisjunction(or);
-        Map<Boolean, List<Expression>> containFuncAndOtherFunc = disjuncts.stream()
-                .collect(Collectors.partitioningBy(this::isValidArrayContains));
-        Map<Expression, Set<Literal>> containLiteralSet = new HashMap<>();
-        List<Expression> contains = containFuncAndOtherFunc.get(true);
-        List<Expression> others = containFuncAndOtherFunc.get(false);
 
-        contains.forEach(containFunc ->
-                containLiteralSet.computeIfAbsent(containFunc.child(0), k -> new HashSet<>())
-                            .add((Literal) containFunc.child(1)));
+        List<Expression> contains = Lists.newArrayList();
+        List<Expression> others = Lists.newArrayList();
+        for (Expression expr : disjuncts) {
+            if (ArrayContainToArrayOverlap.isValidArrayContains(expr)) {
+                contains.add(expr);
+            } else {
+                others.add(expr);
+            }
+        }
+
+        if (contains.size() <= 1) {
+            return or;
+        }
+
+        SetMultimap<Expression, Literal> containLiteralSet = Multimaps.newSetMultimap(
+                new LinkedHashMap<>(), LinkedHashSet::new
+        );
+        for (Expression contain : contains) {
+            containLiteralSet.put(contain.child(0), (Literal) contain.child(1));
+        }
 
         Builder<Expression> newDisjunctsBuilder = new ImmutableList.Builder<>();
-        containLiteralSet.forEach((left, literalSet) -> {
+        for (Entry<Expression, Collection<Literal>> kv : containLiteralSet.asMap().entrySet()) {
+            Expression left = kv.getKey();
+            Collection<Literal> literalSet = kv.getValue();
             if (literalSet.size() > REWRITE_PREDICATE_THRESHOLD) {
                 newDisjunctsBuilder.add(
-                        new ArraysOverlap(left,
-                                new ArrayLiteral(ImmutableList.copyOf(literalSet))));
+                    new ArraysOverlap(left, new ArrayLiteral(Utils.fastToImmutableList(literalSet)))
+                );
+            }
+        }
+
+        for (Expression contain : contains) {
+            if (!canCovertToArrayOverlap(contain, containLiteralSet)) {
+                newDisjunctsBuilder.add(contain);
             }
-        });
-
-        contains.stream()
-                .filter(e -> !canCovertToArrayOverlap(e, containLiteralSet))
-                .forEach(newDisjunctsBuilder::add);
-        others.stream()
-                .map(e -> e.accept(this, null))
-                .forEach(newDisjunctsBuilder::add);
+        }
+        newDisjunctsBuilder.addAll(others);
         return ExpressionUtils.or(newDisjunctsBuilder.build());
     }
 
-    private boolean isValidArrayContains(Expression expression) {
+    private static boolean isValidArrayContains(Expression expression) {
         return expression instanceof ArrayContains && expression.child(1) instanceof Literal;
     }
 
-    private boolean canCovertToArrayOverlap(Expression expression, Map<Expression, Set<Literal>> containLiteralSet) {
-        return expression instanceof ArrayContains
-                && containLiteralSet.getOrDefault(expression.child(0),
-                    new HashSet<>()).size() > REWRITE_PREDICATE_THRESHOLD;
+    private static boolean canCovertToArrayOverlap(
+            Expression expression, SetMultimap<Expression, Literal> containLiteralSet) {
+        if (!(expression instanceof ArrayContains)) {
+            return false;
+        }
+        Set<Literal> containLiteral = containLiteralSet.get(expression.child(0));
+        return containLiteral.size() > REWRITE_PREDICATE_THRESHOLD;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/CaseWhenToIf.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/CaseWhenToIf.java
index 6372338406d..cafb0ecd068 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/CaseWhenToIf.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/CaseWhenToIf.java
@@ -17,25 +17,35 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.CaseWhen;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.WhenClause;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.If;
 import org.apache.doris.nereids.trees.expressions.literal.NullLiteral;
 
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
 /**
  * Rewrite rule to convert CASE WHEN to IF.
  * For example:
  * CASE WHEN a > 1 THEN 1 ELSE 0 END -> IF(a > 1, 1, 0)
  */
-public class CaseWhenToIf extends AbstractExpressionRewriteRule {
+public class CaseWhenToIf implements ExpressionPatternRuleFactory {
 
     public static CaseWhenToIf INSTANCE = new CaseWhenToIf();
 
     @Override
-    public Expression visitCaseWhen(CaseWhen caseWhen, ExpressionRewriteContext context) {
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesTopType(CaseWhen.class).then(CaseWhenToIf::rewrite)
+        );
+    }
+
+    private static Expression rewrite(CaseWhen caseWhen) {
         Expression expr = caseWhen;
         if (caseWhen.getWhenClauses().size() == 1) {
             WhenClause whenClause = caseWhen.getWhenClauses().get(0);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ConvertAggStateCast.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ConvertAggStateCast.java
index e5748eb1d59..23900701553 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ConvertAggStateCast.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ConvertAggStateCast.java
@@ -17,8 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.functions.combinator.StateCombinator;
@@ -30,29 +30,30 @@ import org.apache.doris.nereids.util.TypeCoercionUtils;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.List;
+
 /**
  * Follow legacy planner cast agg_state combinator's children if we need cast it to another agg_state type when insert
  */
-public class ConvertAggStateCast extends AbstractExpressionRewriteRule {
+public class ConvertAggStateCast implements ExpressionPatternRuleFactory {
 
     public static ConvertAggStateCast INSTANCE = new ConvertAggStateCast();
 
     @Override
-    public Expression visitCast(Cast cast, ExpressionRewriteContext context) {
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesTopType(Cast.class).then(ConvertAggStateCast::convert)
+        );
+    }
+
+    private static Expression convert(Cast cast) {
         Expression child = cast.child();
         DataType originalType = child.getDataType();
         DataType targetType = cast.getDataType();
         if (originalType instanceof AggStateType
                 && targetType instanceof AggStateType
                 && child instanceof StateCombinator) {
-            AggStateType original = (AggStateType) originalType;
             AggStateType target = (AggStateType) targetType;
-            if (original.getSubTypes().size() != target.getSubTypes().size()) {
-                return processCastChild(cast, context);
-            }
-            if (!original.getFunctionName().equalsIgnoreCase(target.getFunctionName())) {
-                return processCastChild(cast, context);
-            }
             ImmutableList.Builder<Expression> newChildren = ImmutableList.builderWithExpectedSize(child.arity());
             for (int i = 0; i < child.arity(); i++) {
                 Expression newChild = TypeCoercionUtils.castIfNotSameType(child.child(i), target.getSubTypes().get(i));
@@ -66,15 +67,7 @@ public class ConvertAggStateCast extends AbstractExpressionRewriteRule {
                 newChildren.add(newChild);
             }
             child = child.withChildren(newChildren.build());
-            return processCastChild(cast.withChildren(ImmutableList.of(child)), context);
-        }
-        return processCastChild(cast, context);
-    }
-
-    private Expression processCastChild(Cast cast, ExpressionRewriteContext context) {
-        Expression child = visit(cast.child(), context);
-        if (child != cast.child()) {
-            cast = cast.withChildren(ImmutableList.of(child));
+            return cast.withChildren(ImmutableList.of(child));
         }
         return cast;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DateFunctionRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DateFunctionRewrite.java
index e78eeecff0d..07ec0c3de71 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DateFunctionRewrite.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DateFunctionRewrite.java
@@ -17,8 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.And;
 import org.apache.doris.nereids.trees.expressions.EqualTo;
 import org.apache.doris.nereids.trees.expressions.Expression;
@@ -34,17 +34,31 @@ import org.apache.doris.nereids.trees.expressions.literal.DateV2Literal;
 import org.apache.doris.nereids.types.DateTimeType;
 import org.apache.doris.nereids.types.DateTimeV2Type;
 
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
 /**
  * F: a DateTime or DateTimeV2 column
  * Date(F) > 2020-01-01 => F > 2020-01-02 00:00:00
  * Date(F) >= 2020-01-01 => F > 2020-01-01 00:00:00
  *
  */
-public class DateFunctionRewrite extends AbstractExpressionRewriteRule {
+public class DateFunctionRewrite implements ExpressionPatternRuleFactory {
     public static DateFunctionRewrite INSTANCE = new DateFunctionRewrite();
 
     @Override
-    public Expression visitEqualTo(EqualTo equalTo, ExpressionRewriteContext context) {
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(EqualTo.class).then(DateFunctionRewrite::rewriteEqualTo),
+                matchesType(GreaterThan.class).then(DateFunctionRewrite::rewriteGreaterThan),
+                matchesType(GreaterThanEqual.class).then(DateFunctionRewrite::rewriteGreaterThanEqual),
+                matchesType(LessThan.class).then(DateFunctionRewrite::rewriteLessThan),
+                matchesType(LessThanEqual.class).then(DateFunctionRewrite::rewriteLessThanEqual)
+        );
+    }
+
+    private static Expression rewriteEqualTo(EqualTo equalTo) {
         if (equalTo.left() instanceof Date) {
             // V1
             if (equalTo.left().child(0).getDataType() instanceof DateTimeType
@@ -70,8 +84,7 @@ public class DateFunctionRewrite extends AbstractExpressionRewriteRule {
         return equalTo;
     }
 
-    @Override
-    public Expression visitGreaterThan(GreaterThan greaterThan, ExpressionRewriteContext context) {
+    private static Expression rewriteGreaterThan(GreaterThan greaterThan) {
         if (greaterThan.left() instanceof Date) {
             // V1
             if (greaterThan.left().child(0).getDataType() instanceof DateTimeType
@@ -91,8 +104,7 @@ public class DateFunctionRewrite extends AbstractExpressionRewriteRule {
         return greaterThan;
     }
 
-    @Override
-    public Expression visitGreaterThanEqual(GreaterThanEqual greaterThanEqual, ExpressionRewriteContext context) {
+    private static Expression rewriteGreaterThanEqual(GreaterThanEqual greaterThanEqual) {
         if (greaterThanEqual.left() instanceof Date) {
             // V1
             if (greaterThanEqual.left().child(0).getDataType() instanceof DateTimeType
@@ -111,8 +123,7 @@ public class DateFunctionRewrite extends AbstractExpressionRewriteRule {
         return greaterThanEqual;
     }
 
-    @Override
-    public Expression visitLessThan(LessThan lessThan, ExpressionRewriteContext context) {
+    private static Expression rewriteLessThan(LessThan lessThan) {
         if (lessThan.left() instanceof Date) {
             // V1
             if (lessThan.left().child(0).getDataType() instanceof DateTimeType
@@ -131,8 +142,7 @@ public class DateFunctionRewrite extends AbstractExpressionRewriteRule {
         return lessThan;
     }
 
-    @Override
-    public Expression visitLessThanEqual(LessThanEqual lessThanEqual, ExpressionRewriteContext context) {
+    private static Expression rewriteLessThanEqual(LessThanEqual lessThanEqual) {
         if (lessThanEqual.left() instanceof Date) {
             // V1
             if (lessThanEqual.left().child(0).getDataType() instanceof DateTimeType
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DigitalMaskingConvert.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DigitalMaskingConvert.java
index 5e38c0390b6..95d25e3c592 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DigitalMaskingConvert.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DigitalMaskingConvert.java
@@ -17,8 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Concat;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DigitalMasking;
@@ -26,16 +26,25 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Left;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Right;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
 /**
  * Convert DigitalMasking to Concat
  */
-public class DigitalMaskingConvert extends AbstractExpressionRewriteRule {
-
+public class DigitalMaskingConvert implements ExpressionPatternRuleFactory {
     public static DigitalMaskingConvert INSTANCE = new DigitalMaskingConvert();
 
     @Override
-    public Expression visitDigitalMasking(DigitalMasking digitalMasking, ExpressionRewriteContext context) {
-        return new Concat(new Left(digitalMasking.child(), Literal.of(3)), Literal.of("****"),
-                new Right(digitalMasking.child(), Literal.of(4)));
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(DigitalMasking.class).then(digitalMasking ->
+                    new Concat(
+                            new Left(digitalMasking.child(), Literal.of(3)),
+                            Literal.of("****"),
+                            new Right(digitalMasking.child(), Literal.of(4)))
+                    )
+        );
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DistinctPredicatesRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DistinctPredicatesRule.java
index a3466d395d5..cf18886cd85 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DistinctPredicatesRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/DistinctPredicatesRule.java
@@ -17,12 +17,13 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.CompoundPredicate;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.util.ExpressionUtils;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.util.LinkedHashSet;
@@ -35,16 +36,21 @@ import java.util.Set;
  * transform (a = 1) and (b > 2) and (a = 1)  to (a = 1) and (b > 2)
  * transform (a = 1) or (a = 1) to (a = 1)
  */
-public class DistinctPredicatesRule extends AbstractExpressionRewriteRule {
-
+public class DistinctPredicatesRule implements ExpressionPatternRuleFactory {
     public static final DistinctPredicatesRule INSTANCE = new DistinctPredicatesRule();
 
     @Override
-    public Expression visitCompoundPredicate(CompoundPredicate expr, ExpressionRewriteContext context) {
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesTopType(CompoundPredicate.class).then(DistinctPredicatesRule::distinct)
+        );
+    }
+
+    private static Expression distinct(CompoundPredicate expr) {
         List<Expression> extractExpressions = ExpressionUtils.extract(expr);
         Set<Expression> distinctExpressions = new LinkedHashSet<>(extractExpressions);
         if (distinctExpressions.size() != extractExpressions.size()) {
-            return ExpressionUtils.combine(expr.getClass(), Lists.newArrayList(distinctExpressions));
+            return ExpressionUtils.combineAsLeftDeepTree(expr.getClass(), Lists.newArrayList(distinctExpressions));
         }
         return expr;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ExtractCommonFactorRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ExtractCommonFactorRule.java
index dd457e01d8d..4032db4aadf 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ExtractCommonFactorRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ExtractCommonFactorRule.java
@@ -18,21 +18,28 @@
 package org.apache.doris.nereids.rules.expression.rules;
 
 import org.apache.doris.nereids.annotation.Developing;
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.CompoundPredicate;
 import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral;
 import org.apache.doris.nereids.util.ExpressionUtils;
+import org.apache.doris.nereids.util.Utils;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
 import com.google.common.collect.Sets;
 
-import java.util.Collections;
-import java.util.HashSet;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
+import java.util.Map.Entry;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * Extract common expr for `CompoundPredicate`.
@@ -41,42 +48,197 @@ import java.util.stream.Collectors;
  * transform (a and b) or (a and c) to a and (b or c)
  */
 @Developing
-public class ExtractCommonFactorRule extends AbstractExpressionRewriteRule {
-
+public class ExtractCommonFactorRule implements ExpressionPatternRuleFactory {
     public static final ExtractCommonFactorRule INSTANCE = new ExtractCommonFactorRule();
 
     @Override
-    public Expression visitCompoundPredicate(CompoundPredicate expr, ExpressionRewriteContext context) {
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                 matchesTopType(CompoundPredicate.class).then(ExtractCommonFactorRule::extractCommonFactor)
+        );
+    }
+
+    private static Expression extractCommonFactor(CompoundPredicate originExpr) {
+        // fast return
+        if (!(originExpr.left() instanceof CompoundPredicate || originExpr.left() instanceof BooleanLiteral)
+                && !(originExpr.right() instanceof CompoundPredicate || originExpr.right() instanceof BooleanLiteral)) {
+            return originExpr;
+        }
 
-        Expression rewrittenChildren = ExpressionUtils.combine(expr.getClass(), ExpressionUtils.extract(expr).stream()
-                .map(predicate -> rewrite(predicate, context)).collect(ImmutableList.toImmutableList()));
-        if (!(rewrittenChildren instanceof CompoundPredicate)) {
-            return rewrittenChildren;
+        // flatten same type to a list
+        // e.g. ((a and (b or c)) and c) -> [a, (b or c), c]
+        List<Expression> flatten = ExpressionUtils.extract(originExpr);
+
+        // combine and delete some boolean literal predicate
+        // e.g. (a and true) -> true
+        Expression simplified = ExpressionUtils.combineAsLeftDeepTree(originExpr.getClass(), flatten);
+        if (!(simplified instanceof CompoundPredicate)) {
+            return simplified;
         }
 
-        CompoundPredicate compoundPredicate = (CompoundPredicate) rewrittenChildren;
+        // separate two levels CompoundPredicate to partitions
+        // e.g. ((a and (b or c)) and c) -> [[a], [b, c], c]
+        CompoundPredicate leftDeapTree = (CompoundPredicate) simplified;
+        ImmutableSet.Builder<List<Expression>> partitionsBuilder
+                = ImmutableSet.builderWithExpectedSize(flatten.size());
+        for (Expression onPartition : ExpressionUtils.extract(leftDeapTree)) {
+            if (onPartition instanceof CompoundPredicate) {
+                partitionsBuilder.add(ExpressionUtils.extract((CompoundPredicate) onPartition));
+            } else {
+                partitionsBuilder.add(ImmutableList.of(onPartition));
+            }
+        }
+        Set<List<Expression>> partitions = partitionsBuilder.build();
 
-        List<List<Expression>> partitions = ExpressionUtils.extract(compoundPredicate).stream()
-                .map(predicate -> predicate instanceof CompoundPredicate ? ExpressionUtils.extract(
-                        (CompoundPredicate) predicate) : Lists.newArrayList(predicate)).collect(Collectors.toList());
+        Expression result = extractCommonFactors(originExpr, leftDeapTree, Utils.fastToImmutableList(partitions));
+        return result;
+    }
 
-        Set<Expression> commons = partitions.stream()
-                .<Set<Expression>>map(HashSet::new)
-                .reduce(Sets::intersection)
-                .orElse(Collections.emptySet());
+    private static Expression extractCommonFactors(CompoundPredicate originPredicate,
+            CompoundPredicate leftDeapTreePredicate, List<List<Expression>> initPartitions) {
+        // extract factor and fill into commonFactorToPartIds
+        // e.g.
+        //      originPredicate:         (a and (b and c)) and (b or c)
+        //      leftDeapTreePredicate:   ((a and b) and c) and (b or c)
+        //      initPartitions: [[a], [b], [c], [b, c]]
+        //
+        //   -> commonFactorToPartIds = {a: [0], b: [1, 3], c: [2, 3]}.
+        //      so we can know `b` and `c` is a common factors
+        SetMultimap<Expression, Integer> commonFactorToPartIds = Multimaps.newSetMultimap(
+                Maps.newLinkedHashMap(), LinkedHashSet::new
+        );
+        int originExpressionNum = 0;
+        int partId = 0;
+        for (List<Expression> partition : initPartitions) {
+            for (Expression expression : partition) {
+                commonFactorToPartIds.put(expression, partId);
+                originExpressionNum++;
+            }
+            partId++;
+        }
 
-        List<List<Expression>> uncorrelated = partitions.stream()
-                .map(predicates -> predicates.stream().filter(p -> !commons.contains(p)).collect(Collectors.toList()))
-                .collect(Collectors.toList());
+        //     commonFactorToPartIds = {a: [0], b: [1, 3], c: [2, 3]}
+        //
+        //  -> reverse key value of commonFactorToPartIds and remove intersecting partition
+        //
+        //  -> 1. reverse: {[0]: [a], [1, 3]: [b], [2, 3]: [c]}
+        //  -> 2. sort by key size desc: {[1, 3]: [b], [2, 3]: [c], [0]: [a]}
+        //  -> 3. remove intersection partition: {[1, 3]: [b], [2]: [c], [0]: [a]},
+        //        because first part and second part intersect by partition 3
+        LinkedHashMap<Set<Integer>, Set<Expression>> commonFactorPartitions
+                = partitionByMostCommonFactors(commonFactorToPartIds);
 
-        Expression combineUncorrelated = ExpressionUtils.combine(compoundPredicate.getClass(),
-                uncorrelated.stream()
-                        .map(predicates -> ExpressionUtils.combine(compoundPredicate.flipType(), predicates))
-                        .collect(Collectors.toList()));
+        int extractedExpressionNum = 0;
+        for (Set<Expression> exprs : commonFactorPartitions.values()) {
+            extractedExpressionNum += exprs.size();
+        }
+
+        // no any common factor
+        if (commonFactorPartitions.entrySet().iterator().next().getKey().size() <= 1
+                && !(originPredicate.getWidth() > leftDeapTreePredicate.getWidth())
+                && originExpressionNum <= extractedExpressionNum) {
+            // this condition is important because it can avoid deap loop:
+            // origin originExpr:               A = 1 and (B > 0 and B < 10)
+            // after ExtractCommonFactorRule:   (A = 1 and B > 0) and (B < 10)     (left deap tree)
+            // after SimplifyRange:             A = 1 and (B > 0 and B < 10)       (right deap tree)
+            return originPredicate;
+        }
+
+        // now we can do extract common factors for each part:
+        //    originPredicate:         (a and (b and c)) and (b or c)
+        //    leftDeapTreePredicate:   ((a and b) and c) and (b or c)
+        //    initPartitions:          [[a], [b], [c], [b, c]]
+        //    commonFactorPartitions:  {[1, 3]: [b], [0]: [a]}
+        //
+        // -> extractedExprs: [
+        //                       b or (false and c) = b,
+        //                       a,
+        //                       c
+        //                    ]
+        //
+        // -> result: (b or c) and a and c
+        ImmutableList.Builder<Expression> extractedExprs
+                = ImmutableList.builderWithExpectedSize(commonFactorPartitions.size());
+        for (Entry<Set<Integer>, Set<Expression>> kv : commonFactorPartitions.entrySet()) {
+            Expression extracted = doExtractCommonFactors(
+                    leftDeapTreePredicate, initPartitions, kv.getKey(), kv.getValue()
+            );
+            extractedExprs.add(extracted);
+        }
+
+        // combine and eliminate some boolean literal predicate
+        return ExpressionUtils.combineAsLeftDeepTree(leftDeapTreePredicate.getClass(), extractedExprs.build());
+    }
 
-        List<Expression> finalCompound = Lists.newArrayList(commons);
-        finalCompound.add(combineUncorrelated);
+    private static Expression doExtractCommonFactors(
+            CompoundPredicate originPredicate,
+            List<List<Expression>> partitions, Set<Integer> partitionIds, Set<Expression> commonFactors) {
+        ImmutableList.Builder<Expression> uncorrelatedExprPartitionsBuilder
+                = ImmutableList.builderWithExpectedSize(partitionIds.size());
+        for (Integer partitionId : partitionIds) {
+            List<Expression> partition = partitions.get(partitionId);
+            ImmutableSet.Builder<Expression> uncorrelatedBuilder
+                    = ImmutableSet.builderWithExpectedSize(partition.size());
+            for (Expression exprOfPart : partition) {
+                if (!commonFactors.contains(exprOfPart)) {
+                    uncorrelatedBuilder.add(exprOfPart);
+                }
+            }
+
+            Set<Expression> uncorrelated = uncorrelatedBuilder.build();
+            Expression partitionWithoutCommonFactor
+                    = ExpressionUtils.combineAsLeftDeepTree(originPredicate.flipType(), uncorrelated);
+            if (partitionWithoutCommonFactor instanceof CompoundPredicate) {
+                partitionWithoutCommonFactor = extractCommonFactor((CompoundPredicate) partitionWithoutCommonFactor);
+            }
+            uncorrelatedExprPartitionsBuilder.add(partitionWithoutCommonFactor);
+        }
+
+        ImmutableList<Expression> uncorrelatedExprPartitions = uncorrelatedExprPartitionsBuilder.build();
+        ImmutableList.Builder<Expression> allExprs = ImmutableList.builderWithExpectedSize(commonFactors.size() + 1);
+        allExprs.addAll(commonFactors);
+
+        Expression combineUncorrelatedExpr = ExpressionUtils.combineAsLeftDeepTree(
+                originPredicate.getClass(), uncorrelatedExprPartitions);
+        allExprs.add(combineUncorrelatedExpr);
+
+        Expression result = ExpressionUtils.combineAsLeftDeepTree(originPredicate.flipType(), allExprs.build());
+        return result;
+    }
+
+    private static LinkedHashMap<Set<Integer>, Set<Expression>> partitionByMostCommonFactors(
+            SetMultimap<Expression, Integer> commonFactorToPartIds) {
+        SetMultimap<Set<Integer>, Expression> partWithCommonFactors = Multimaps.newSetMultimap(
+                Maps.newLinkedHashMap(), LinkedHashSet::new
+        );
+
+        for (Entry<Expression, Collection<Integer>> factorToId : commonFactorToPartIds.asMap().entrySet()) {
+            partWithCommonFactors.put((Set<Integer>) factorToId.getValue(), factorToId.getKey());
+        }
+
+        List<Set<Integer>> sortedPartitionIdHasCommonFactor = Lists.newArrayList(partWithCommonFactors.keySet());
+        // place the most common factor at the head of this list
+        sortedPartitionIdHasCommonFactor.sort((p1, p2) -> p2.size() - p1.size());
+
+        LinkedHashMap<Set<Integer>, Set<Expression>> shouldExtractFactors = Maps.newLinkedHashMap();
+
+        Set<Integer> allocatedPartitions = Sets.newLinkedHashSet();
+        for (Set<Integer> originMostCommonFactorPartitions : sortedPartitionIdHasCommonFactor) {
+            ImmutableSet.Builder<Integer> notAllocatePartitions = ImmutableSet.builderWithExpectedSize(
+                    originMostCommonFactorPartitions.size());
+            for (Integer partId : originMostCommonFactorPartitions) {
+                if (allocatedPartitions.add(partId)) {
+                    notAllocatePartitions.add(partId);
+                }
+            }
+
+            Set<Integer> mostCommonFactorPartitions = notAllocatePartitions.build();
+            if (!mostCommonFactorPartitions.isEmpty()) {
+                Set<Expression> commonFactors = partWithCommonFactors.get(originMostCommonFactorPartitions);
+                shouldExtractFactors.put(mostCommonFactorPartitions, commonFactors);
+            }
+        }
 
-        return ExpressionUtils.combine(compoundPredicate.flipType(), finalCompound);
+        return shouldExtractFactors;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRule.java
index c801f749ee0..04acb91d9e2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRule.java
@@ -17,24 +17,46 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
+import org.apache.doris.nereids.rules.expression.ExpressionBottomUpRewriter;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
+import org.apache.doris.nereids.rules.expression.ExpressionRewrite;
 import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
 import org.apache.doris.nereids.trees.expressions.Expression;
 
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
 /**
  * Constant evaluation of an expression.
  */
-public class FoldConstantRule extends AbstractExpressionRewriteRule {
+public class FoldConstantRule implements ExpressionPatternRuleFactory {
 
     public static final FoldConstantRule INSTANCE = new FoldConstantRule();
 
+    private static final ExpressionBottomUpRewriter FULL_FOLD_REWRITER = ExpressionRewrite.bottomUp(
+            FoldConstantRuleOnFE.VISITOR_INSTANCE,
+            FoldConstantRuleOnBE.INSTANCE
+    );
+
+    /** evaluate by pattern match */
     @Override
-    public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) {
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.<ExpressionPatternMatcher<? extends Expression>>builder()
+                .addAll(FoldConstantRuleOnFE.PATTERN_MATCH_INSTANCE.buildRules())
+                .addAll(FoldConstantRuleOnBE.INSTANCE.buildRules())
+                .build();
+    }
+
+    /** evaluate by visitor */
+    public static Expression evaluate(Expression expr, ExpressionRewriteContext ctx) {
         if (ctx.cascadesContext != null
                 && ctx.cascadesContext.getConnectContext() != null
                 && ctx.cascadesContext.getConnectContext().getSessionVariable().isEnableFoldConstantByBe()) {
-            return new FoldConstantRuleOnBE().rewrite(expr, ctx);
+            return FULL_FOLD_REWRITER.rewrite(expr, ctx);
+        } else {
+            return FoldConstantRuleOnFE.VISITOR_INSTANCE.rewrite(expr, ctx);
         }
-        return FoldConstantRuleOnFE.INSTANCE.rewrite(expr, ctx);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java
index 38c6a483c9f..09e9bbe0b91 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java
@@ -27,8 +27,9 @@ import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.DebugUtil;
 import org.apache.doris.common.util.TimeUtils;
 import org.apache.doris.nereids.glue.translator.ExpressionTranslator;
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionMatchingContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Alias;
 import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.Expression;
@@ -55,6 +56,7 @@ import org.apache.doris.thrift.TQueryGlobals;
 import org.apache.doris.thrift.TQueryOptions;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -73,24 +75,38 @@ import java.util.concurrent.TimeUnit;
 /**
  * Constant evaluation of an expression.
  */
-public class FoldConstantRuleOnBE extends AbstractExpressionRewriteRule {
+public class FoldConstantRuleOnBE implements ExpressionPatternRuleFactory {
+
+    public static final FoldConstantRuleOnBE INSTANCE = new FoldConstantRuleOnBE();
     private static final Logger LOG = LogManager.getLogger(FoldConstantRuleOnBE.class);
-    private final IdGenerator<ExprId> idGenerator = ExprId.createGenerator();
 
     @Override
-    public Expression rewrite(Expression expression, ExpressionRewriteContext ctx) {
-        expression = FoldConstantRuleOnFE.INSTANCE.rewrite(expression, ctx);
-        return foldByBE(expression, ctx);
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                root(Expression.class)
+                    .whenCtx(FoldConstantRuleOnBE::isEnableFoldByBe)
+                    .thenApply(FoldConstantRuleOnBE::foldByBE)
+        );
+    }
+
+    public static boolean isEnableFoldByBe(ExpressionMatchingContext<Expression> ctx) {
+        return ctx.cascadesContext != null
+                && ctx.cascadesContext.getConnectContext() != null
+                && ctx.cascadesContext.getConnectContext().getSessionVariable().isEnableFoldConstantByBe();
     }
 
-    private Expression foldByBE(Expression root, ExpressionRewriteContext context) {
+    /** foldByBE */
+    public static Expression foldByBE(ExpressionMatchingContext<Expression> context) {
+        IdGenerator<ExprId> idGenerator = ExprId.createGenerator();
+
+        Expression root = context.expr;
         Map<String, Expression> constMap = Maps.newHashMap();
         Map<String, TExpr> staleConstTExprMap = Maps.newHashMap();
         Expression rootWithoutAlias = root;
         if (root instanceof Alias) {
             rootWithoutAlias = ((Alias) root).child();
         }
-        collectConst(rootWithoutAlias, constMap, staleConstTExprMap);
+        collectConst(rootWithoutAlias, constMap, staleConstTExprMap, idGenerator);
         if (constMap.isEmpty()) {
             return root;
         }
@@ -103,7 +119,8 @@ public class FoldConstantRuleOnBE extends AbstractExpressionRewriteRule {
         return root;
     }
 
-    private Expression replace(Expression root, Map<String, Expression> constMap, Map<String, Expression> resultMap) {
+    private static Expression replace(
+            Expression root, Map<String, Expression> constMap, Map<String, Expression> resultMap) {
         for (Entry<String, Expression> entry : constMap.entrySet()) {
             if (entry.getValue().equals(root)) {
                 return resultMap.get(entry.getKey());
@@ -121,7 +138,8 @@ public class FoldConstantRuleOnBE extends AbstractExpressionRewriteRule {
         return hasNewChildren ? root.withChildren(newChildren) : root;
     }
 
-    private void collectConst(Expression expr, Map<String, Expression> constMap, Map<String, TExpr> tExprMap) {
+    private static void collectConst(Expression expr, Map<String, Expression> constMap,
+            Map<String, TExpr> tExprMap, IdGenerator<ExprId> idGenerator) {
         if (expr.isConstant()) {
             // Do not constant fold cast(null as dataType) because we cannot preserve the
             // cast-to-types and that can lead to query failures, e.g., CTAS
@@ -157,13 +175,13 @@ public class FoldConstantRuleOnBE extends AbstractExpressionRewriteRule {
         } else {
             for (int i = 0; i < expr.children().size(); i++) {
                 final Expression child = expr.children().get(i);
-                collectConst(child, constMap, tExprMap);
+                collectConst(child, constMap, tExprMap, idGenerator);
             }
         }
     }
 
     // if sleep(5) will cause rpc timeout
-    private boolean skipSleepFunction(Expression expr) {
+    private static boolean skipSleepFunction(Expression expr) {
         if (expr instanceof Sleep) {
             Expression param = expr.child(0);
             if (param instanceof Cast) {
@@ -176,7 +194,7 @@ public class FoldConstantRuleOnBE extends AbstractExpressionRewriteRule {
         return false;
     }
 
-    private Map<String, Expression> evalOnBE(Map<String, Map<String, TExpr>> paramMap,
+    private static Map<String, Expression> evalOnBE(Map<String, Map<String, TExpr>> paramMap,
             Map<String, Expression> constMap, ConnectContext context) {
 
         Map<String, Expression> resultMap = new HashMap<>();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java
index 05165f6c312..cf3d1a88d8c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java
@@ -22,7 +22,13 @@ import org.apache.doris.catalog.Env;
 import org.apache.doris.cluster.ClusterNamespace;
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
+import org.apache.doris.nereids.rules.expression.ExpressionListenerMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionMatchingContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionTraverseListener;
+import org.apache.doris.nereids.rules.expression.ExpressionTraverseListenerFactory;
 import org.apache.doris.nereids.trees.expressions.AggregateExpression;
 import org.apache.doris.nereids.trees.expressions.And;
 import org.apache.doris.nereids.trees.expressions.BinaryArithmetic;
@@ -80,6 +86,8 @@ import org.apache.doris.qe.GlobalVariable;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableList.Builder;
 import com.google.common.collect.Lists;
 import org.apache.commons.codec.digest.DigestUtils;
 
@@ -87,13 +95,78 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
 
 /**
  * evaluate an expression on fe.
  */
-public class FoldConstantRuleOnFE extends AbstractExpressionRewriteRule {
+public class FoldConstantRuleOnFE extends AbstractExpressionRewriteRule
+        implements ExpressionPatternRuleFactory, ExpressionTraverseListenerFactory {
 
-    public static final FoldConstantRuleOnFE INSTANCE = new FoldConstantRuleOnFE();
+    public static final FoldConstantRuleOnFE VISITOR_INSTANCE = new FoldConstantRuleOnFE(true);
+    public static final FoldConstantRuleOnFE PATTERN_MATCH_INSTANCE = new FoldConstantRuleOnFE(false);
+
+    // record whether current expression is in an aggregate function with distinct,
+    // if is, we will skip to fold constant
+    private static final ListenAggDistinct LISTEN_AGG_DISTINCT = new ListenAggDistinct();
+    private static final CheckWhetherUnderAggDistinct NOT_UNDER_AGG_DISTINCT = new CheckWhetherUnderAggDistinct();
+
+    private final boolean deepRewrite;
+
+    public FoldConstantRuleOnFE(boolean deepRewrite) {
+        this.deepRewrite = deepRewrite;
+    }
+
+    public static Expression evaluate(Expression expression, ExpressionRewriteContext expressionRewriteContext) {
+        return VISITOR_INSTANCE.rewrite(expression, expressionRewriteContext);
+    }
+
+    @Override
+    public List<ExpressionListenerMatcher<? extends Expression>> buildListeners() {
+        return ImmutableList.of(
+                listenerType(AggregateFunction.class)
+                        .when(AggregateFunction::isDistinct)
+                        .then(LISTEN_AGG_DISTINCT.as()),
+
+                listenerType(AggregateExpression.class)
+                        .when(AggregateExpression::isDistinct)
+                        .then(LISTEN_AGG_DISTINCT.as())
+        );
+    }
+
+    @Override
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matches(EncryptKeyRef.class, this::visitEncryptKeyRef),
+                matches(EqualTo.class, this::visitEqualTo),
+                matches(GreaterThan.class, this::visitGreaterThan),
+                matches(GreaterThanEqual.class, this::visitGreaterThanEqual),
+                matches(LessThan.class, this::visitLessThan),
+                matches(LessThanEqual.class, this::visitLessThanEqual),
+                matches(NullSafeEqual.class, this::visitNullSafeEqual),
+                matches(Not.class, this::visitNot),
+                matches(Database.class, this::visitDatabase),
+                matches(CurrentUser.class, this::visitCurrentUser),
+                matches(CurrentCatalog.class, this::visitCurrentCatalog),
+                matches(User.class, this::visitUser),
+                matches(ConnectionId.class, this::visitConnectionId),
+                matches(And.class, this::visitAnd),
+                matches(Or.class, this::visitOr),
+                matches(Cast.class, this::visitCast),
+                matches(BoundFunction.class, this::visitBoundFunction),
+                matches(BinaryArithmetic.class, this::visitBinaryArithmetic),
+                matches(CaseWhen.class, this::visitCaseWhen),
+                matches(If.class, this::visitIf),
+                matches(InPredicate.class, this::visitInPredicate),
+                matches(IsNull.class, this::visitIsNull),
+                matches(TimestampArithmetic.class, this::visitTimestampArithmetic),
+                matches(Password.class, this::visitPassword),
+                matches(Array.class, this::visitArray),
+                matches(Date.class, this::visitDate),
+                matches(Version.class, this::visitVersion)
+        );
+    }
 
     @Override
     public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) {
@@ -253,7 +326,7 @@ public class FoldConstantRuleOnFE extends AbstractExpressionRewriteRule {
         List<Expression> nonTrueLiteral = Lists.newArrayList();
         int nullCount = 0;
         for (Expression e : and.children()) {
-            e = e.accept(this, context);
+            e = deepRewrite ? e.accept(this, context) : e;
             if (BooleanLiteral.FALSE.equals(e)) {
                 return BooleanLiteral.FALSE;
             } else if (e instanceof NullLiteral) {
@@ -294,7 +367,7 @@ public class FoldConstantRuleOnFE extends AbstractExpressionRewriteRule {
         List<Expression> nonFalseLiteral = Lists.newArrayList();
         int nullCount = 0;
         for (Expression e : or.children()) {
-            e = e.accept(this, context);
+            e = deepRewrite ? e.accept(this, context) : e;
             if (BooleanLiteral.TRUE.equals(e)) {
                 return BooleanLiteral.TRUE;
             } else if (e instanceof NullLiteral) {
@@ -412,9 +485,13 @@ public class FoldConstantRuleOnFE extends AbstractExpressionRewriteRule {
             }
         }
 
-        Expression defaultResult = caseWhen.getDefaultValue().isPresent()
-                ? rewrite(caseWhen.getDefaultValue().get(), context)
-                : null;
+        Expression defaultResult = null;
+        if (caseWhen.getDefaultValue().isPresent()) {
+            defaultResult = caseWhen.getDefaultValue().get();
+            if (deepRewrite) {
+                defaultResult = rewrite(defaultResult, context);
+            }
+        }
         if (foundNewDefault) {
             defaultResult = newDefault;
         }
@@ -537,28 +614,83 @@ public class FoldConstantRuleOnFE extends AbstractExpressionRewriteRule {
         return new StringLiteral(GlobalVariable.version);
     }
 
-    private <E> E rewriteChildren(Expression expr, ExpressionRewriteContext ctx) {
-        return (E) super.visit(expr, ctx);
-    }
-
-    private boolean allArgsIsAllLiteral(Expression expression) {
-        return ExpressionUtils.isAllLiteral(expression.getArguments());
-    }
-
-    private boolean argsHasNullLiteral(Expression expression) {
-        return ExpressionUtils.hasNullLiteral(expression.getArguments());
+    private <E extends Expression> E rewriteChildren(E expr, ExpressionRewriteContext context) {
+        if (!deepRewrite) {
+            return expr;
+        }
+        switch (expr.arity()) {
+            case 1: {
+                Expression originChild = expr.child(0);
+                Expression newChild = originChild.accept(this, context);
+                return (originChild != newChild) ? (E) expr.withChildren(ImmutableList.of(newChild)) : expr;
+            }
+            case 2: {
+                Expression originLeft = expr.child(0);
+                Expression newLeft = originLeft.accept(this, context);
+                Expression originRight = expr.child(1);
+                Expression newRight = originRight.accept(this, context);
+                return (originLeft != newLeft || originRight != newRight)
+                        ? (E) expr.withChildren(ImmutableList.of(newLeft, newRight))
+                        : expr;
+            }
+            case 0: {
+                return expr;
+            }
+            default: {
+                boolean hasNewChildren = false;
+                Builder<Expression> newChildren = ImmutableList.builderWithExpectedSize(expr.arity());
+                for (Expression child : expr.children()) {
+                    Expression newChild = child.accept(this, context);
+                    if (newChild != child) {
+                        hasNewChildren = true;
+                    }
+                    newChildren.add(newChild);
+                }
+                return hasNewChildren ? (E) expr.withChildren(newChildren.build()) : expr;
+            }
+        }
     }
 
     private Optional<Expression> preProcess(Expression expression) {
         if (expression instanceof AggregateFunction || expression instanceof TableGeneratingFunction) {
             return Optional.of(expression);
         }
-        if (expression instanceof PropagateNullable && argsHasNullLiteral(expression)) {
+        if (expression instanceof PropagateNullable && ExpressionUtils.hasNullLiteral(expression.getArguments())) {
             return Optional.of(new NullLiteral(expression.getDataType()));
         }
-        if (!allArgsIsAllLiteral(expression)) {
+        if (!ExpressionUtils.isAllLiteral(expression.getArguments())) {
             return Optional.of(expression);
         }
         return Optional.empty();
     }
+
+    private static class ListenAggDistinct implements ExpressionTraverseListener<Expression> {
+        @Override
+        public void onEnter(ExpressionMatchingContext<Expression> context) {
+            context.cascadesContext.incrementDistinctAggLevel();
+        }
+
+        @Override
+        public void onExit(ExpressionMatchingContext<Expression> context, Expression rewritten) {
+            context.cascadesContext.decrementDistinctAggLevel();
+        }
+    }
+
+    private static class CheckWhetherUnderAggDistinct implements Predicate<ExpressionMatchingContext<Expression>> {
+        @Override
+        public boolean test(ExpressionMatchingContext<Expression> context) {
+            return context.cascadesContext.getDistinctAggLevel() == 0;
+        }
+
+        public <E extends Expression> Predicate<ExpressionMatchingContext<E>> as() {
+            return (Predicate) this;
+        }
+    }
+
+    private <E extends Expression> ExpressionPatternMatcher<? extends Expression> matches(
+            Class<E> clazz, BiFunction<E, ExpressionRewriteContext, Expression> visitMethod) {
+        return matchesType(clazz)
+                .whenCtx(NOT_UNDER_AGG_DISTINCT.as())
+                .thenApply(ctx -> visitMethod.apply(ctx.expr, ctx.rewriteContext));
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/InPredicateDedup.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/InPredicateDedup.java
index 32f8e46da75..3760dcf0e72 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/InPredicateDedup.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/InPredicateDedup.java
@@ -17,13 +17,14 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.InPredicate;
 
-import java.util.ArrayList;
-import java.util.HashSet;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
 import java.util.List;
 import java.util.Set;
 
@@ -31,25 +32,32 @@ import java.util.Set;
  * Deduplicate InPredicate, For example:
  * where A in (x, x) ==> where A in (x)
  */
-public class InPredicateDedup extends AbstractExpressionRewriteRule {
-
-    public static InPredicateDedup INSTANCE = new InPredicateDedup();
+public class InPredicateDedup implements ExpressionPatternRuleFactory {
+    public static final InPredicateDedup INSTANCE = new InPredicateDedup();
 
     @Override
-    public Expression visitInPredicate(InPredicate inPredicate, ExpressionRewriteContext context) {
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+            matchesType(InPredicate.class).then(InPredicateDedup::dedup)
+        );
+    }
+
+    /** dedup */
+    public static Expression dedup(InPredicate inPredicate) {
         // In many BI scenarios, the sql is auto-generated, and hence there may be thousands of options.
         // It takes a long time to apply this rule. So set a threshold for the max number.
-        if (inPredicate.getOptions().size() > 200) {
+        int optionSize = inPredicate.getOptions().size();
+        if (optionSize > 200) {
             return inPredicate;
         }
-        Set<Expression> dedupExpr = new HashSet<>();
-        List<Expression> newOptions = new ArrayList<>();
+        ImmutableSet.Builder<Expression> newOptionsBuilder = ImmutableSet.builderWithExpectedSize(inPredicate.arity());
         for (Expression option : inPredicate.getOptions()) {
-            if (dedupExpr.contains(option)) {
-                continue;
-            }
-            dedupExpr.add(option);
-            newOptions.add(option);
+            newOptionsBuilder.add(option);
+        }
+
+        Set<Expression> newOptions = newOptionsBuilder.build();
+        if (newOptions.size() == optionSize) {
+            return inPredicate;
         }
         return new InPredicate(inPredicate.getCompareExpr(), newOptions);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/InPredicateToEqualToRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/InPredicateToEqualToRule.java
index b076cadd533..353de7f41f6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/InPredicateToEqualToRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/InPredicateToEqualToRule.java
@@ -17,12 +17,14 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.EqualTo;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.InPredicate;
 
+import com.google.common.collect.ImmutableList;
+
 import java.util.List;
 
 /**
@@ -36,17 +38,16 @@ import java.util.List;
  * NOTICE: it's related with `SimplifyRange`.
  * They are same processes, so must change synchronously.
  */
-public class InPredicateToEqualToRule extends AbstractExpressionRewriteRule {
-
-    public static InPredicateToEqualToRule INSTANCE = new InPredicateToEqualToRule();
+public class InPredicateToEqualToRule implements ExpressionPatternRuleFactory {
+    public static final InPredicateToEqualToRule INSTANCE = new InPredicateToEqualToRule();
 
     @Override
-    public Expression visitInPredicate(InPredicate inPredicate, ExpressionRewriteContext context) {
-        Expression left = inPredicate.getCompareExpr();
-        List<Expression> right = inPredicate.getOptions();
-        if (right.size() != 1) {
-            return new InPredicate(left.accept(this, context), right);
-        }
-        return new EqualTo(left.accept(this, context), right.get(0).accept(this, context));
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(InPredicate.class)
+                    .when(in -> in.getOptions().size() == 1)
+                    .then(in -> new EqualTo(in.getCompareExpr(), in.getOptions().get(0))
+                )
+        );
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/NormalizeBinaryPredicatesRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/NormalizeBinaryPredicatesRule.java
index 9b1c88b930b..e73104793cd 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/NormalizeBinaryPredicatesRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/NormalizeBinaryPredicatesRule.java
@@ -17,22 +17,31 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
 import org.apache.doris.nereids.trees.expressions.Expression;
 
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
 /**
  * Normalizes binary predicates of the form 'expr' op 'slot' so that the slot is on the left-hand side.
  * For example:
  * 5 > id -> id < 5
  */
-public class NormalizeBinaryPredicatesRule extends AbstractExpressionRewriteRule {
-
-    public static NormalizeBinaryPredicatesRule INSTANCE = new NormalizeBinaryPredicatesRule();
+public class NormalizeBinaryPredicatesRule implements ExpressionPatternRuleFactory {
+    public static final NormalizeBinaryPredicatesRule INSTANCE = new NormalizeBinaryPredicatesRule();
 
     @Override
-    public Expression visitComparisonPredicate(ComparisonPredicate expr, ExpressionRewriteContext context) {
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(ComparisonPredicate.class).then(NormalizeBinaryPredicatesRule::normalize)
+        );
+    }
+
+    public static Expression normalize(ComparisonPredicate expr) {
         return expr.left().isConstant() && !expr.right().isConstant() ? expr.commute() : expr;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/NullSafeEqualToEqual.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/NullSafeEqualToEqual.java
index 6507f49825c..e8eedb1e198 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/NullSafeEqualToEqual.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/NullSafeEqualToEqual.java
@@ -17,31 +17,34 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteRule;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.EqualTo;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.IsNull;
 import org.apache.doris.nereids.trees.expressions.NullSafeEqual;
 import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.NullLiteral;
-import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
 
 /**
  * convert "<=>" to "=", if any side is not nullable
  * convert "A <=> null" to "A is null"
  */
-public class NullSafeEqualToEqual extends DefaultExpressionRewriter<ExpressionRewriteContext> implements
-        ExpressionRewriteRule<ExpressionRewriteContext> {
+public class NullSafeEqualToEqual implements ExpressionPatternRuleFactory {
     public static final NullSafeEqualToEqual INSTANCE = new NullSafeEqualToEqual();
 
     @Override
-    public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) {
-        return expr.accept(this, null);
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(NullSafeEqual.class).then(NullSafeEqualToEqual::rewrite)
+        );
     }
 
-    @Override
-    public Expression visitNullSafeEqual(NullSafeEqual nullSafeEqual, ExpressionRewriteContext ctx) {
+    private static Expression rewrite(NullSafeEqual nullSafeEqual) {
         if (nullSafeEqual.left() instanceof NullLiteral) {
             if (nullSafeEqual.right().nullable()) {
                 return new IsNull(nullSafeEqual.right());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneListPartitionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneListPartitionEvaluator.java
index dd71ed8e99f..b9bdf520e3d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneListPartitionEvaluator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneListPartitionEvaluator.java
@@ -82,7 +82,7 @@ public class OneListPartitionEvaluator
         expr = super.visit(expr, context);
         if (!(expr instanceof Literal)) {
             // just forward to fold constant rule
-            return expr.accept(FoldConstantRuleOnFE.INSTANCE, expressionRewriteContext);
+            return FoldConstantRuleOnFE.evaluate(expr, expressionRewriteContext);
         }
         return expr;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java
index 4e2ba5be909..2c0f8c13939 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OneRangePartitionEvaluator.java
@@ -92,7 +92,7 @@ public class OneRangePartitionEvaluator
 
     /** OneRangePartitionEvaluator */
     public OneRangePartitionEvaluator(long partitionId, List<Slot> partitionSlots,
-            RangePartitionItem partitionItem, CascadesContext cascadesContext) {
+            RangePartitionItem partitionItem, CascadesContext cascadesContext, int expandThreshold) {
         this.partitionId = partitionId;
         this.partitionSlots = Objects.requireNonNull(partitionSlots, "partitionSlots cannot be null");
         this.partitionItem = Objects.requireNonNull(partitionItem, "partitionItem cannot be null");
@@ -103,41 +103,46 @@ public class OneRangePartitionEvaluator
         this.lowers = toNereidsLiterals(range.lowerEndpoint());
         this.uppers = toNereidsLiterals(range.upperEndpoint());
 
-        PartitionRangeExpander expander = new PartitionRangeExpander();
-        this.partitionSlotTypes = expander.computePartitionSlotTypes(lowers, uppers);
-        this.slotToType = Maps.newHashMapWithExpectedSize(16);
-        for (int i = 0; i < partitionSlots.size(); i++) {
-            slotToType.put(partitionSlots.get(i), partitionSlotTypes.get(i));
-        }
+        this.partitionSlotTypes = PartitionRangeExpander.computePartitionSlotTypes(lowers, uppers);
 
-        this.partitionSlotContainsNull = Maps.newHashMapWithExpectedSize(16);
-        for (int i = 0; i < partitionSlots.size(); i++) {
-            Slot slot = partitionSlots.get(i);
-            if (!slot.nullable()) {
-                partitionSlotContainsNull.put(slot, false);
-                continue;
+        if (partitionSlots.size() == 1) {
+            // fast path
+            Slot partSlot = partitionSlots.get(0);
+            this.slotToType = ImmutableMap.of(partSlot, partitionSlotTypes.get(0));
+            this.partitionSlotContainsNull
+                    = ImmutableMap.of(partSlot, range.lowerEndpoint().getKeys().get(0).isMinValue());
+        } else {
+            // slow path
+            this.slotToType = Maps.newHashMap();
+            for (int i = 0; i < partitionSlots.size(); i++) {
+                slotToType.put(partitionSlots.get(i), partitionSlotTypes.get(i));
             }
-            PartitionSlotType partitionSlotType = partitionSlotTypes.get(i);
-            boolean maybeNull = false;
-            switch (partitionSlotType) {
-                case CONST:
-                case RANGE:
-                    maybeNull = range.lowerEndpoint().getKeys().get(i).isMinValue();
-                    break;
-                case OTHER:
-                    maybeNull = true;
-                    break;
-                default:
-                    throw new AnalysisException("Unknown partition slot type: " + partitionSlotType);
+
+            this.partitionSlotContainsNull = Maps.newHashMap();
+            for (int i = 0; i < partitionSlots.size(); i++) {
+                Slot slot = partitionSlots.get(i);
+                if (!slot.nullable()) {
+                    partitionSlotContainsNull.put(slot, false);
+                    continue;
+                }
+                PartitionSlotType partitionSlotType = partitionSlotTypes.get(i);
+                boolean maybeNull;
+                switch (partitionSlotType) {
+                    case CONST:
+                    case RANGE:
+                        maybeNull = range.lowerEndpoint().getKeys().get(i).isMinValue();
+                        break;
+                    case OTHER:
+                        maybeNull = true;
+                        break;
+                    default:
+                        throw new AnalysisException("Unknown partition slot type: " + partitionSlotType);
+                }
+                partitionSlotContainsNull.put(slot, maybeNull);
             }
-            partitionSlotContainsNull.put(slot, maybeNull);
         }
 
-        int expandThreshold = cascadesContext.getAndCacheSessionVariable(
-                "partitionPruningExpandThreshold",
-                10, sessionVariable -> sessionVariable.partitionPruningExpandThreshold);
-
-        List<List<Expression>> expandInputs = expander.tryExpandRange(
+        List<List<Expression>> expandInputs = PartitionRangeExpander.tryExpandRange(
                 partitionSlots, lowers, uppers, partitionSlotTypes, expandThreshold);
         // after expand range, we will get 2 dimension list like list:
         // part_col1: [1], part_col2:[4, 5, 6], we should combine it to
@@ -451,10 +456,13 @@ public class OneRangePartitionEvaluator
 
     private EvaluateRangeResult evaluateChildrenThenThis(Expression expr, EvaluateRangeInput context) {
         // evaluate children
-        List<Expression> newChildren = new ArrayList<>();
-        List<EvaluateRangeResult> childrenResults = new ArrayList<>();
+        List<Expression> children = expr.children();
+        ImmutableList.Builder<Expression> newChildren = ImmutableList.builderWithExpectedSize(children.size());
+        List<EvaluateRangeResult> childrenResults = new ArrayList<>(children.size());
         boolean hasNewChildren = false;
-        for (Expression child : expr.children()) {
+
+        for (int i = 0; i < children.size(); i++) {
+            Expression child = children.get(i);
             EvaluateRangeResult childResult = child.accept(this, context);
             if (childResult.result != child) {
                 hasNewChildren = true;
@@ -463,11 +471,11 @@ public class OneRangePartitionEvaluator
             newChildren.add(childResult.result);
         }
         if (hasNewChildren) {
-            expr = expr.withChildren(newChildren);
+            expr = expr.withChildren(newChildren.build());
         }
 
         // evaluate this
-        expr = expr.accept(FoldConstantRuleOnFE.INSTANCE, expressionRewriteContext);
+        expr = FoldConstantRuleOnFE.evaluate(expr, expressionRewriteContext);
         return new EvaluateRangeResult(expr, context.defaultColumnRanges, childrenResults);
     }
 
@@ -575,9 +583,28 @@ public class OneRangePartitionEvaluator
     }
 
     private List<Literal> toNereidsLiterals(PartitionKey partitionKey) {
-        List<Literal> literals = Lists.newArrayListWithCapacity(partitionKey.getKeys().size());
-        for (int i = 0; i < partitionKey.getKeys().size(); i++) {
-            LiteralExpr literalExpr = partitionKey.getKeys().get(i);
+        if (partitionKey.getKeys().size() == 1) {
+            // fast path
+            return toSingleNereidsLiteral(partitionKey);
+        }
+
+        // slow path
+        return toMultiNereidsLiterals(partitionKey);
+    }
+
+    private List<Literal> toSingleNereidsLiteral(PartitionKey partitionKey) {
+        List<LiteralExpr> keys = partitionKey.getKeys();
+        LiteralExpr literalExpr = keys.get(0);
+        PrimitiveType primitiveType = partitionKey.getTypes().get(0);
+        Type type = Type.fromPrimitiveType(primitiveType);
+        return ImmutableList.of(Literal.fromLegacyLiteral(literalExpr, type));
+    }
+
+    private List<Literal> toMultiNereidsLiterals(PartitionKey partitionKey) {
+        List<LiteralExpr> keys = partitionKey.getKeys();
+        List<Literal> literals = Lists.newArrayListWithCapacity(keys.size());
+        for (int i = 0; i < keys.size(); i++) {
+            LiteralExpr literalExpr = keys.get(i);
             PrimitiveType primitiveType = partitionKey.getTypes().get(i);
             Type type = Type.fromPrimitiveType(primitiveType);
             literals.add(Literal.fromLegacyLiteral(literalExpr, type));
@@ -613,8 +640,8 @@ public class OneRangePartitionEvaluator
         Literal lower = span.lowerEndpoint().getValue();
         Literal upper = span.upperEndpoint().getValue();
 
-        Expression lowerDate = new Date(lower).accept(FoldConstantRuleOnFE.INSTANCE, expressionRewriteContext);
-        Expression upperDate = new Date(upper).accept(FoldConstantRuleOnFE.INSTANCE, expressionRewriteContext);
+        Expression lowerDate = FoldConstantRuleOnFE.evaluate(new Date(lower), expressionRewriteContext);
+        Expression upperDate = FoldConstantRuleOnFE.evaluate(new Date(upper), expressionRewriteContext);
 
         if (lowerDate instanceof Literal && upperDate instanceof Literal && lowerDate.equals(upperDate)) {
             return new EvaluateRangeResult(lowerDate, result.columnRanges, result.childrenResult);
@@ -696,7 +723,7 @@ public class OneRangePartitionEvaluator
 
         public EvaluateRangeResult(Expression result, Map<Slot, ColumnRange> columnRanges,
                 List<EvaluateRangeResult> childrenResult) {
-            this(result, columnRanges, childrenResult, childrenResult.stream().allMatch(r -> r.isRejectNot()));
+            this(result, columnRanges, childrenResult, allIsRejectNot(childrenResult));
         }
 
         public EvaluateRangeResult withRejectNot(boolean rejectNot) {
@@ -706,6 +733,15 @@ public class OneRangePartitionEvaluator
         public boolean isRejectNot() {
             return rejectNot;
         }
+
+        private static boolean allIsRejectNot(List<EvaluateRangeResult> childrenResult) {
+            for (EvaluateRangeResult evaluateRangeResult : childrenResult) {
+                if (!evaluateRangeResult.isRejectNot()) {
+                    return false;
+                }
+            }
+            return true;
+        }
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OrToIn.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OrToIn.java
index b085f70da6e..83da8055037 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OrToIn.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/OrToIn.java
@@ -17,15 +17,17 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
+import org.apache.doris.nereids.rules.expression.ExpressionBottomUpRewriter;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
+import org.apache.doris.nereids.rules.expression.ExpressionRewrite;
 import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteRule;
 import org.apache.doris.nereids.trees.expressions.EqualTo;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.InPredicate;
 import org.apache.doris.nereids.trees.expressions.NamedExpression;
 import org.apache.doris.nereids.trees.expressions.Or;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
-import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter;
 import org.apache.doris.nereids.util.ExpressionUtils;
 
 import com.google.common.collect.ImmutableList;
@@ -54,20 +56,25 @@ import java.util.Set;
  * adding any additional rule-specific fields to the default ExpressionRewriteContext. However, the entire expression
  * rewrite framework always passes an ExpressionRewriteContext of type context to all rules.
  */
-public class OrToIn extends DefaultExpressionRewriter<ExpressionRewriteContext> implements
-        ExpressionRewriteRule<ExpressionRewriteContext> {
+public class OrToIn implements ExpressionPatternRuleFactory {
 
     public static final OrToIn INSTANCE = new OrToIn();
 
     public static final int REWRITE_OR_TO_IN_PREDICATE_THRESHOLD = 2;
 
     @Override
-    public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) {
-        return expr.accept(this, null);
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesTopType(Or.class).then(OrToIn::rewrite)
+        );
     }
 
-    @Override
-    public Expression visitOr(Or or, ExpressionRewriteContext ctx) {
+    public Expression rewriteTree(Expression expr, ExpressionRewriteContext context) {
+        ExpressionBottomUpRewriter bottomUpRewriter = ExpressionRewrite.bottomUp(this);
+        return bottomUpRewriter.rewrite(expr, context);
+    }
+
+    private static Expression rewrite(Or or) {
         // NOTICE: use linked hash map to avoid unstable order or entry.
         //  unstable order entry lead to dead loop since return expression always un-equals to original one.
         Map<NamedExpression, Set<Literal>> slotNameToLiteral = Maps.newLinkedHashMap();
@@ -80,6 +87,10 @@ public class OrToIn extends DefaultExpressionRewriter<ExpressionRewriteContext>
                 handleInPredicate((InPredicate) expression, slotNameToLiteral, disConjunctToSlot);
             }
         }
+        if (disConjunctToSlot.isEmpty()) {
+            return or;
+        }
+
         List<Expression> rewrittenOr = new ArrayList<>();
         for (Map.Entry<NamedExpression, Set<Literal>> entry : slotNameToLiteral.entrySet()) {
             Set<Literal> literals = entry.getValue();
@@ -90,7 +101,7 @@ public class OrToIn extends DefaultExpressionRewriter<ExpressionRewriteContext>
         }
         for (Expression expression : expressions) {
             if (disConjunctToSlot.get(expression) == null) {
-                rewrittenOr.add(expression.accept(this, null));
+                rewrittenOr.add(expression);
             } else {
                 Set<Literal> literals = slotNameToLiteral.get(disConjunctToSlot.get(expression));
                 if (literals.size() < REWRITE_OR_TO_IN_PREDICATE_THRESHOLD) {
@@ -102,7 +113,7 @@ public class OrToIn extends DefaultExpressionRewriter<ExpressionRewriteContext>
         return ExpressionUtils.or(rewrittenOr);
     }
 
-    private void handleEqualTo(EqualTo equal, Map<NamedExpression, Set<Literal>> slotNameToLiteral,
+    private static void handleEqualTo(EqualTo equal, Map<NamedExpression, Set<Literal>> slotNameToLiteral,
                                Map<Expression, NamedExpression> disConjunctToSlot) {
         Expression left = equal.left();
         Expression right = equal.right();
@@ -115,7 +126,7 @@ public class OrToIn extends DefaultExpressionRewriter<ExpressionRewriteContext>
         }
     }
 
-    private void handleInPredicate(InPredicate inPredicate, Map<NamedExpression, Set<Literal>> slotNameToLiteral,
+    private static void handleInPredicate(InPredicate inPredicate, Map<NamedExpression, Set<Literal>> slotNameToLiteral,
                                    Map<Expression, NamedExpression> disConjunctToSlot) {
         // TODO a+b in (1,2,3...) is not supported now
         if (inPredicate.getCompareExpr() instanceof NamedExpression
@@ -127,10 +138,9 @@ public class OrToIn extends DefaultExpressionRewriter<ExpressionRewriteContext>
         }
     }
 
-    public void addSlotToLiteral(NamedExpression namedExpression, Literal literal,
+    private static void addSlotToLiteral(NamedExpression namedExpression, Literal literal,
             Map<NamedExpression, Set<Literal>> slotNameToLiteral) {
         Set<Literal> literals = slotNameToLiteral.computeIfAbsent(namedExpression, k -> new LinkedHashSet<>());
         literals.add(literal);
     }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java
index b8440777872..04cef999fda 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionPruner.java
@@ -21,6 +21,7 @@ import org.apache.doris.catalog.ListPartitionItem;
 import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.RangePartitionItem;
 import org.apache.doris.nereids.CascadesContext;
+import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
 import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
 import org.apache.doris.nereids.trees.expressions.Expression;
@@ -81,14 +82,19 @@ public class PartitionPruner extends DefaultExpressionRewriter<Void> {
                 && ((Cast) right).child().getDataType().isDateType()) {
             DateTimeLiteral dt = (DateTimeLiteral) left;
             Cast cast = (Cast) right;
-            return cp.withChildren(new DateLiteral(dt.getYear(), dt.getMonth(), dt.getDay()), cast.child());
+            return cp.withChildren(
+                    ImmutableList.of(new DateLiteral(dt.getYear(), dt.getMonth(), dt.getDay()), cast.child())
+            );
         } else if (right instanceof DateTimeLiteral && ((DateTimeLiteral) right).isMidnight()
                 && left instanceof Cast
                 && ((Cast) left).child() instanceof SlotReference
                 && ((Cast) left).child().getDataType().isDateType()) {
             DateTimeLiteral dt = (DateTimeLiteral) right;
             Cast cast = (Cast) left;
-            return cp.withChildren(cast.child(), new DateLiteral(dt.getYear(), dt.getMonth(), dt.getDay()));
+            return cp.withChildren(ImmutableList.of(
+                    cast.child(),
+                    new DateLiteral(dt.getYear(), dt.getMonth(), dt.getDay()))
+            );
         } else {
             return cp;
         }
@@ -115,13 +121,18 @@ public class PartitionPruner extends DefaultExpressionRewriter<Void> {
                 partitionPredicate, ImmutableSet.copyOf(partitionSlots), cascadesContext);
         partitionPredicate = PredicateRewriteForPartitionPrune.rewrite(partitionPredicate, cascadesContext);
 
+        int expandThreshold = cascadesContext.getAndCacheSessionVariable(
+                "partitionPruningExpandThreshold",
+                10, sessionVariable -> sessionVariable.partitionPruningExpandThreshold);
+
         List<OnePartitionEvaluator> evaluators = Lists.newArrayListWithCapacity(idToPartitions.size());
         for (Entry<Long, PartitionItem> kv : idToPartitions.entrySet()) {
             evaluators.add(toPartitionEvaluator(
-                    kv.getKey(), kv.getValue(), partitionSlots, cascadesContext, partitionTableType));
+                    kv.getKey(), kv.getValue(), partitionSlots, cascadesContext, expandThreshold));
         }
 
-        partitionPredicate = OrToIn.INSTANCE.rewrite(partitionPredicate, null);
+        partitionPredicate = OrToIn.INSTANCE.rewriteTree(
+                partitionPredicate, new ExpressionRewriteContext(cascadesContext));
         PartitionPruner partitionPruner = new PartitionPruner(evaluators, partitionPredicate);
         //TODO: we keep default partition because it's too hard to prune it, we return false in canPrune().
         return partitionPruner.prune();
@@ -131,13 +142,13 @@ public class PartitionPruner extends DefaultExpressionRewriter<Void> {
      * convert partition item to partition evaluator
      */
     public static final OnePartitionEvaluator toPartitionEvaluator(long id, PartitionItem partitionItem,
-            List<Slot> partitionSlots, CascadesContext cascadesContext, PartitionTableType partitionTableType) {
+            List<Slot> partitionSlots, CascadesContext cascadesContext, int expandThreshold) {
         if (partitionItem instanceof ListPartitionItem) {
             return new OneListPartitionEvaluator(
                     id, partitionSlots, (ListPartitionItem) partitionItem, cascadesContext);
         } else if (partitionItem instanceof RangePartitionItem) {
             return new OneRangePartitionEvaluator(
-                    id, partitionSlots, (RangePartitionItem) partitionItem, cascadesContext);
+                    id, partitionSlots, (RangePartitionItem) partitionItem, cascadesContext, expandThreshold);
         } else {
             return new UnknownPartitionEvaluator(id, partitionItem);
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionRangeExpander.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionRangeExpander.java
index 071ab8f1157..01a674488da 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionRangeExpander.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PartitionRangeExpander.java
@@ -41,7 +41,6 @@ import java.time.LocalDate;
 import java.time.ZoneOffset;
 import java.util.Iterator;
 import java.util.List;
-import java.util.NoSuchElementException;
 import java.util.function.Function;
 
 /**
@@ -74,10 +73,44 @@ public class PartitionRangeExpander {
     }
 
     /** expandRangeLiterals */
-    public final List<List<Expression>> tryExpandRange(
+    public static final List<List<Expression>> tryExpandRange(
             List<Slot> partitionSlots, List<Literal> lowers, List<Literal> uppers,
             List<PartitionSlotType> partitionSlotTypes, int expandThreshold) {
+        if (partitionSlots.size() == 1) {
+            return tryExpandSingleColumnRange(partitionSlots.get(0), lowers.get(0),
+                    uppers.get(0), expandThreshold);
+        } else {
+            // slow path
+            return commonTryExpandRange(partitionSlots, lowers, uppers, partitionSlotTypes, expandThreshold);
+        }
+    }
+
+    private static List<List<Expression>> tryExpandSingleColumnRange(Slot partitionSlot, Literal lower,
+            Literal upper, int expandThreshold) {
+        // must be range slot
+        try {
+            if (canExpandRange(partitionSlot, lower, upper, 1, expandThreshold)) {
+                Iterator<? extends Expression> iterator = enumerableIterator(
+                        partitionSlot, lower, upper, true);
+                if (iterator instanceof SingletonIterator) {
+                    return ImmutableList.of(ImmutableList.of(iterator.next()));
+                } else {
+                    return ImmutableList.of(
+                            ImmutableList.copyOf(iterator)
+                    );
+                }
+            } else {
+                return ImmutableList.of(ImmutableList.of(partitionSlot));
+            }
+        } catch (Throwable t) {
+            // catch for safety, should not invoke here
+            return ImmutableList.of(ImmutableList.of(partitionSlot));
+        }
+    }
 
+    private static List<List<Expression>> commonTryExpandRange(
+            List<Slot> partitionSlots, List<Literal> lowers, List<Literal> uppers,
+            List<PartitionSlotType> partitionSlotTypes, int expandThreshold) {
         long expandedCount = 1;
         List<List<Expression>> expandedLists = Lists.newArrayListWithCapacity(lowers.size());
         for (int i = 0; i < partitionSlotTypes.size(); i++) {
@@ -126,7 +159,7 @@ public class PartitionRangeExpander {
         return expandedLists;
     }
 
-    private boolean canExpandRange(Slot slot, Literal lower, Literal upper,
+    private static boolean canExpandRange(Slot slot, Literal lower, Literal upper,
             long expandedCount, int expandThreshold) {
         DataType type = slot.getDataType();
         if (!type.isIntegerLikeType() && !type.isDateType() && !type.isDateV2Type()) {
@@ -139,7 +172,7 @@ public class PartitionRangeExpander {
             }
             // too much expanded will consuming resources of frontend,
             // e.g. [1, 100000000), we should skip expand it
-            return (expandedCount * count) <= expandThreshold;
+            return count == 1 || (expandedCount * count) <= expandThreshold;
         } catch (Throwable t) {
             // e.g. max_value can not expand
             return false;
@@ -147,7 +180,7 @@ public class PartitionRangeExpander {
     }
 
     /** the types will like this: [CONST, CONST, ..., RANGE, OTHER, OTHER, ...] */
-    public List<PartitionSlotType> computePartitionSlotTypes(List<Literal> lowers, List<Literal> uppers) {
+    public static List<PartitionSlotType> computePartitionSlotTypes(List<Literal> lowers, List<Literal> uppers) {
         PartitionSlotType previousType = PartitionSlotType.CONST;
         List<PartitionSlotType> types = Lists.newArrayListWithCapacity(lowers.size());
         for (int i = 0; i < lowers.size(); ++i) {
@@ -167,7 +200,7 @@ public class PartitionRangeExpander {
         return types;
     }
 
-    private long enumerableCount(DataType dataType, Literal startInclusive, Literal endExclusive) {
+    private static long enumerableCount(DataType dataType, Literal startInclusive, Literal endExclusive) {
         if (dataType.isIntegerLikeType()) {
             BigInteger start = new BigInteger(startInclusive.getStringValue());
             BigInteger end = new BigInteger(endExclusive.getStringValue());
@@ -175,6 +208,12 @@ public class PartitionRangeExpander {
         } else if (dataType.isDateType()) {
             DateLiteral startInclusiveDate = (DateLiteral) startInclusive;
             DateLiteral endExclusiveDate = (DateLiteral) endExclusive;
+
+            if (startInclusiveDate.getYear() == endExclusiveDate.getYear()
+                    && startInclusiveDate.getMonth() == endExclusiveDate.getMonth()) {
+                return endExclusiveDate.getDay() - startInclusiveDate.getDay();
+            }
+
             LocalDate startDate = LocalDate.of(
                     (int) startInclusiveDate.getYear(),
                     (int) startInclusiveDate.getMonth(),
@@ -192,6 +231,12 @@ public class PartitionRangeExpander {
         } else if (dataType.isDateV2Type()) {
             DateV2Literal startInclusiveDate = (DateV2Literal) startInclusive;
             DateV2Literal endExclusiveDate = (DateV2Literal) endExclusive;
+
+            if (startInclusiveDate.getYear() == endExclusiveDate.getYear()
+                    && startInclusiveDate.getMonth() == endExclusiveDate.getMonth()) {
+                return endExclusiveDate.getDay() - startInclusiveDate.getDay();
+            }
+
             LocalDate startDate = LocalDate.of(
                     (int) startInclusiveDate.getYear(),
                     (int) startInclusiveDate.getMonth(),
@@ -212,7 +257,7 @@ public class PartitionRangeExpander {
         return -1;
     }
 
-    private Iterator<? extends Expression> enumerableIterator(
+    private static Iterator<? extends Expression> enumerableIterator(
             Slot slot, Literal startInclusive, Literal endLiteral, boolean endExclusive) {
         DataType dataType = slot.getDataType();
         if (dataType.isIntegerLikeType()) {
@@ -237,6 +282,12 @@ public class PartitionRangeExpander {
         } else if (dataType.isDateType()) {
             DateLiteral startInclusiveDate = (DateLiteral) startInclusive;
             DateLiteral endLiteralDate = (DateLiteral) endLiteral;
+            if (endExclusive && startInclusiveDate.getYear() == endLiteralDate.getYear()
+                    && startInclusiveDate.getMonth() == endLiteralDate.getMonth()
+                    && startInclusiveDate.getDay() + 1 == endLiteralDate.getDay()) {
+                return new SingletonIterator(startInclusive);
+            }
+
             LocalDate startDate = LocalDate.of(
                     (int) startInclusiveDate.getYear(),
                     (int) startInclusiveDate.getMonth(),
@@ -258,6 +309,13 @@ public class PartitionRangeExpander {
         } else if (dataType.isDateV2Type()) {
             DateV2Literal startInclusiveDate = (DateV2Literal) startInclusive;
             DateV2Literal endLiteralDate = (DateV2Literal) endLiteral;
+
+            if (endExclusive && startInclusiveDate.getYear() == endLiteralDate.getYear()
+                    && startInclusiveDate.getMonth() == endLiteralDate.getMonth()
+                    && startInclusiveDate.getDay() + 1 == endLiteralDate.getDay()) {
+                return new SingletonIterator(startInclusive);
+            }
+
             LocalDate startDate = LocalDate.of(
                     (int) startInclusiveDate.getYear(),
                     (int) startInclusiveDate.getMonth(),
@@ -282,7 +340,7 @@ public class PartitionRangeExpander {
         return Iterators.singletonIterator(slot);
     }
 
-    private class IntegerLikeRangePartitionValueIterator<L extends IntegerLikeLiteral>
+    private static class IntegerLikeRangePartitionValueIterator<L extends IntegerLikeLiteral>
             extends RangePartitionValueIterator<BigInteger, L> {
 
         public IntegerLikeRangePartitionValueIterator(BigInteger startInclusive, BigInteger end,
@@ -296,7 +354,7 @@ public class PartitionRangeExpander {
         }
     }
 
-    private class DateLikeRangePartitionValueIterator<L extends Literal>
+    private static class DateLikeRangePartitionValueIterator<L extends Literal>
             extends RangePartitionValueIterator<LocalDate, L> {
 
         public DateLikeRangePartitionValueIterator(
@@ -309,43 +367,4 @@ public class PartitionRangeExpander {
             return current.plusDays(1);
         }
     }
-
-    private abstract class RangePartitionValueIterator<C extends Comparable, L extends Literal>
-            implements Iterator<L> {
-        private final C startInclusive;
-        private final C end;
-        private final boolean endExclusive;
-        private C current;
-
-        private final Function<C, L> toLiteral;
-
-        public RangePartitionValueIterator(C startInclusive, C end, boolean endExclusive, Function<C, L> toLiteral) {
-            this.startInclusive = startInclusive;
-            this.end = end;
-            this.endExclusive = endExclusive;
-            this.current = this.startInclusive;
-            this.toLiteral = toLiteral;
-        }
-
-        @Override
-        public boolean hasNext() {
-            if (endExclusive) {
-                return current.compareTo(end) < 0;
-            } else {
-                return current.compareTo(end) <= 0;
-            }
-        }
-
-        @Override
-        public L next() {
-            if (hasNext()) {
-                C value = current;
-                current = doGetNext(current);
-                return toLiteral.apply(value);
-            }
-            throw new NoSuchElementException();
-        }
-
-        protected abstract C doGetNext(C current);
-    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PredicateRewriteForPartitionPrune.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PredicateRewriteForPartitionPrune.java
index c227c89b939..87646fbd582 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PredicateRewriteForPartitionPrune.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/PredicateRewriteForPartitionPrune.java
@@ -70,7 +70,7 @@ public class PredicateRewriteForPartitionPrune
                     }
                 }
                 if (convertable) {
-                    Expression or = ExpressionUtils.combine(Or.class, splitIn);
+                    Expression or = ExpressionUtils.combineAsLeftDeepTree(Or.class, splitIn);
                     return or;
                 }
             } else if (dateChild.getDataType() instanceof DateTimeV2Type) {
@@ -87,7 +87,7 @@ public class PredicateRewriteForPartitionPrune
                     }
                 }
                 if (convertable) {
-                    Expression or = ExpressionUtils.combine(Or.class, splitIn);
+                    Expression or = ExpressionUtils.combineAsLeftDeepTree(Or.class, splitIn);
                     return or;
                 }
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/RangePartitionValueIterator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/RangePartitionValueIterator.java
new file mode 100644
index 00000000000..79ee33d1ebb
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/RangePartitionValueIterator.java
@@ -0,0 +1,64 @@
+// 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.expression.rules;
+
+import org.apache.doris.nereids.trees.expressions.literal.Literal;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.function.Function;
+
+/** RangePartitionValueIterator */
+public abstract class RangePartitionValueIterator<C extends Comparable, L extends Literal>
+        implements Iterator<L> {
+    private final C startInclusive;
+    private final C end;
+    private final boolean endExclusive;
+    private C current;
+
+    private final Function<C, L> toLiteral;
+
+    public RangePartitionValueIterator(C startInclusive, C end, boolean endExclusive, Function<C, L> toLiteral) {
+        this.startInclusive = startInclusive;
+        this.end = end;
+        this.endExclusive = endExclusive;
+        this.current = this.startInclusive;
+        this.toLiteral = toLiteral;
+    }
+
+    @Override
+    public boolean hasNext() {
+        if (endExclusive) {
+            return current.compareTo(end) < 0;
+        } else {
+            return current.compareTo(end) <= 0;
+        }
+    }
+
+    @Override
+    public L next() {
+        if (hasNext()) {
+            C value = current;
+            current = doGetNext(current);
+            return toLiteral.apply(value);
+        }
+        throw new NoSuchElementException();
+    }
+
+    protected abstract C doGetNext(C current);
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java
index 3fd5330395e..b4c5552706c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java
@@ -17,20 +17,25 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.Variable;
 
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
 /**
  * replace varaible to real expression
  */
-public class ReplaceVariableByLiteral extends AbstractExpressionRewriteRule {
-
+public class ReplaceVariableByLiteral implements ExpressionPatternRuleFactory {
     public static ReplaceVariableByLiteral INSTANCE = new ReplaceVariableByLiteral();
 
     @Override
-    public Expression visitVariable(Variable variable, ExpressionRewriteContext context) {
-        return variable.getRealExpression();
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+            matchesType(Variable.class).then(Variable::getRealExpression)
+        );
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticComparisonRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticComparisonRule.java
index 7606d082479..6d18bc7b380 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticComparisonRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticComparisonRule.java
@@ -17,7 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
 import org.apache.doris.nereids.trees.expressions.Add;
 import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
@@ -43,6 +44,7 @@ import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.util.TypeCoercionUtils;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
 import java.util.Arrays;
@@ -55,11 +57,11 @@ import javax.annotation.Nullable;
  * a + 1 > 1 => a > 0
  * a / -2 > 1 => a < -2
  */
-public class SimplifyArithmeticComparisonRule extends AbstractExpressionRewriteRule {
-    public static final SimplifyArithmeticComparisonRule INSTANCE = new SimplifyArithmeticComparisonRule();
+public class SimplifyArithmeticComparisonRule implements ExpressionPatternRuleFactory {
+    public static SimplifyArithmeticComparisonRule INSTANCE = new SimplifyArithmeticComparisonRule();
 
     // don't rearrange multiplication because divide may loss precision
-    final Map<Class<? extends Expression>, Class<? extends Expression>> rearrangementMap = ImmutableMap
+    private static final Map<Class<? extends Expression>, Class<? extends Expression>> REARRANGEMENT_MAP = ImmutableMap
             .<Class<? extends Expression>, Class<? extends Expression>>builder()
             .put(Add.class, Subtract.class)
             .put(Subtract.class, Add.class)
@@ -81,41 +83,54 @@ public class SimplifyArithmeticComparisonRule extends AbstractExpressionRewriteR
             .build();
 
     @Override
-    public Expression visitComparisonPredicate(ComparisonPredicate comparison, ExpressionRewriteContext context) {
-        if (couldRearrange(comparison)) {
-            ComparisonPredicate newComparison = normalize(comparison);
-            if (newComparison == null) {
-                return comparison;
-            }
-            try {
-                List<Expression> children =
-                        tryRearrangeChildren(newComparison.left(), newComparison.right(), context);
-                newComparison = (ComparisonPredicate) visitComparisonPredicate(
-                        (ComparisonPredicate) newComparison.withChildren(children), context);
-            } catch (Exception e) {
-                return comparison;
-            }
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(ComparisonPredicate.class)
+                        .thenApply(ctx -> simplify(ctx.expr, new ExpressionRewriteContext(ctx.cascadesContext)))
+        );
+    }
+
+    /** simplify */
+    public static Expression simplify(ComparisonPredicate comparison, ExpressionRewriteContext context) {
+        if (!couldRearrange(comparison)) {
+            return comparison;
+        }
+        ComparisonPredicate newComparison = normalize(comparison);
+        if (newComparison == null) {
+            return comparison;
+        }
+        try {
+            List<Expression> children = tryRearrangeChildren(newComparison.left(), newComparison.right(), context);
+            newComparison = (ComparisonPredicate) simplify(
+                    (ComparisonPredicate) newComparison.withChildren(children), context);
             return TypeCoercionUtils.processComparisonPredicate(newComparison);
-        } else {
+        } catch (Exception e) {
             return comparison;
         }
     }
 
-    private boolean couldRearrange(ComparisonPredicate cmp) {
-        return rearrangementMap.containsKey(cmp.left().getClass())
-                && !cmp.left().isConstant()
-                && cmp.left().children().stream().anyMatch(Expression::isConstant);
+    private static boolean couldRearrange(ComparisonPredicate cmp) {
+        if (!REARRANGEMENT_MAP.containsKey(cmp.left().getClass()) || cmp.left().isConstant()) {
+            return false;
+        }
+
+        for (Expression child : cmp.left().children()) {
+            if (child.isConstant()) {
+                return true;
+            }
+        }
+        return false;
     }
 
-    private List<Expression> tryRearrangeChildren(Expression left, Expression right,
+    private static List<Expression> tryRearrangeChildren(Expression left, Expression right,
             ExpressionRewriteContext context) throws Exception {
         if (!left.child(1).isConstant()) {
             throw new RuntimeException(String.format("Expected literal when arranging children for Expr %s", left));
         }
-        Literal leftLiteral = (Literal) FoldConstantRule.INSTANCE.rewrite(left.child(1), context);
+        Literal leftLiteral = (Literal) FoldConstantRule.evaluate(left.child(1), context);
         Expression leftExpr = left.child(0);
 
-        Class<? extends Expression> oppositeOperator = rearrangementMap.get(left.getClass());
+        Class<? extends Expression> oppositeOperator = REARRANGEMENT_MAP.get(left.getClass());
         Expression newChild = oppositeOperator.getConstructor(Expression.class, Expression.class)
                 .newInstance(right, leftLiteral);
 
@@ -127,25 +142,25 @@ public class SimplifyArithmeticComparisonRule extends AbstractExpressionRewriteR
     }
 
     // Ensure that the second child must be Literal, such as
-    private @Nullable ComparisonPredicate normalize(ComparisonPredicate comparison) {
-        if (!(comparison.left().child(1) instanceof Literal)) {
-            Expression left = comparison.left();
-            if (comparison.left() instanceof Add) {
-                // 1 + a > 1 => a + 1 > 1
-                Expression newLeft = left.withChildren(left.child(1), left.child(0));
-                comparison = (ComparisonPredicate) comparison.withChildren(newLeft, comparison.right());
-            } else if (comparison.left() instanceof Subtract) {
-                // 1 - a > 1 => a + 1 < 1
-                Expression newLeft = left.child(0);
-                Expression newRight = new Add(left.child(1), comparison.right());
-                comparison = (ComparisonPredicate) comparison.withChildren(newLeft, newRight);
-                comparison = comparison.commute();
-            } else {
-                // Don't normalize division/multiplication because the slot sign is undecided.
-                return null;
-            }
+    private static @Nullable ComparisonPredicate normalize(ComparisonPredicate comparison) {
+        Expression left = comparison.left();
+        Expression leftRight = left.child(1);
+        if (leftRight instanceof Literal) {
+            return comparison;
+        }
+        if (left instanceof Add) {
+            // 1 + a > 1 => a + 1 > 1
+            Expression newLeft = left.withChildren(leftRight, left.child(0));
+            return (ComparisonPredicate) comparison.withChildren(newLeft, comparison.right());
+        } else if (left instanceof Subtract) {
+            // 1 - a > 1 => a + 1 < 1
+            Expression newLeft = left.child(0);
+            Expression newRight = new Add(leftRight, comparison.right());
+            comparison = (ComparisonPredicate) comparison.withChildren(newLeft, newRight);
+            return comparison.commute();
+        } else {
+            // Don't normalize division/multiplication because the slot sign is undecided.
+            return null;
         }
-        return comparison;
     }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticRule.java
index fc7431a9994..b9fd91f6438 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyArithmeticRule.java
@@ -17,8 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Add;
 import org.apache.doris.nereids.trees.expressions.BinaryArithmetic;
 import org.apache.doris.nereids.trees.expressions.Divide;
@@ -27,7 +27,9 @@ import org.apache.doris.nereids.trees.expressions.Multiply;
 import org.apache.doris.nereids.trees.expressions.Subtract;
 import org.apache.doris.nereids.util.TypeCoercionUtils;
 import org.apache.doris.nereids.util.TypeUtils;
+import org.apache.doris.nereids.util.Utils;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.util.List;
@@ -43,27 +45,24 @@ import java.util.Optional;
  *
  * TODO: handle cases like: '1 - IA < 1' to 'IA > 0'
  */
-public class SimplifyArithmeticRule extends AbstractExpressionRewriteRule {
+public class SimplifyArithmeticRule implements ExpressionPatternRuleFactory {
     public static final SimplifyArithmeticRule INSTANCE = new SimplifyArithmeticRule();
 
     @Override
-    public Expression visitAdd(Add add, ExpressionRewriteContext context) {
-        return process(add, true);
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesTopType(BinaryArithmetic.class).then(SimplifyArithmeticRule::simplify)
+        );
     }
 
-    @Override
-    public Expression visitSubtract(Subtract subtract, ExpressionRewriteContext context) {
-        return process(subtract, true);
-    }
-
-    @Override
-    public Expression visitDivide(Divide divide, ExpressionRewriteContext context) {
-        return process(divide, false);
-    }
-
-    @Override
-    public Expression visitMultiply(Multiply multiply, ExpressionRewriteContext context) {
-        return process(multiply, false);
+    /** simplify */
+    public static Expression simplify(BinaryArithmetic binaryArithmetic) {
+        if (binaryArithmetic instanceof Add || binaryArithmetic instanceof Subtract) {
+            return process(binaryArithmetic, true);
+        } else if (binaryArithmetic instanceof Multiply || binaryArithmetic instanceof Divide) {
+            return process(binaryArithmetic, false);
+        }
+        return binaryArithmetic;
     }
 
     /**
@@ -75,7 +74,7 @@ public class SimplifyArithmeticRule extends AbstractExpressionRewriteRule {
      * 3.build new arithmetic expression.
      *   (a + b - c + d) + (1 - 2 - 1)
      */
-    private Expression process(BinaryArithmetic arithmetic, boolean isAddOrSub) {
+    private static Expression process(BinaryArithmetic arithmetic, boolean isAddOrSub) {
         // 1. flatten the arithmetic expression.
         List<Operand> flattedExpressions = flatten(arithmetic, isAddOrSub);
 
@@ -83,22 +82,24 @@ public class SimplifyArithmeticRule extends AbstractExpressionRewriteRule {
         List<Operand> constants = Lists.newArrayList();
 
         // TODO currently we don't process decimal for simplicity.
-        if (flattedExpressions.stream().anyMatch(operand -> operand.expression.getDataType().isDecimalLikeType())) {
-            return arithmetic;
+        for (Operand operand : flattedExpressions) {
+            if (operand.expression.getDataType().isDecimalLikeType()) {
+                return arithmetic;
+            }
         }
         // 2. move variables to left side and move constants to right sid.
-        flattedExpressions.forEach(operand -> {
+        for (Operand operand : flattedExpressions) {
             if (operand.expression.isConstant()) {
                 constants.add(operand);
             } else {
                 variables.add(operand);
             }
-        });
+        }
 
         // 3. build new arithmetic expression.
         if (!constants.isEmpty()) {
             boolean isOpposite = !constants.get(0).flag;
-            Optional<Operand> c = constants.stream().reduce((x, y) -> {
+            Optional<Operand> c = Utils.fastReduce(constants, (x, y) -> {
                 Expression expr;
                 if (isOpposite && y.flag || !isOpposite && !y.flag) {
                     expr = getSubOrDivide(isAddOrSub, x, y);
@@ -115,10 +116,10 @@ public class SimplifyArithmeticRule extends AbstractExpressionRewriteRule {
             }
         }
 
-        Optional<Operand> result = variables.stream().reduce((x, y) -> !y.flag
+        Optional<Operand> result = Utils.fastReduce(variables, (x, y) -> !y.flag
                 ? Operand.of(true, getSubOrDivide(isAddOrSub, x, y))
-                : Operand.of(true, getAddOrMultiply(isAddOrSub, x, y)));
-
+                : Operand.of(true, getAddOrMultiply(isAddOrSub, x, y))
+        );
         if (result.isPresent()) {
             return TypeCoercionUtils.castIfNotSameType(result.get().expression, arithmetic.getDataType());
         } else {
@@ -126,7 +127,7 @@ public class SimplifyArithmeticRule extends AbstractExpressionRewriteRule {
         }
     }
 
-    private List<Operand> flatten(Expression expr, boolean isAddOrSub) {
+    private static List<Operand> flatten(Expression expr, boolean isAddOrSub) {
         List<Operand> result = Lists.newArrayList();
         if (isAddOrSub) {
             flattenAddSubtract(true, expr, result);
@@ -136,7 +137,7 @@ public class SimplifyArithmeticRule extends AbstractExpressionRewriteRule {
         return result;
     }
 
-    private void flattenAddSubtract(boolean flag, Expression expr, List<Operand> result) {
+    private static void flattenAddSubtract(boolean flag, Expression expr, List<Operand> result) {
         if (TypeUtils.isAddOrSubtract(expr)) {
             BinaryArithmetic arithmetic = (BinaryArithmetic) expr;
             flattenAddSubtract(flag, arithmetic.left(), result);
@@ -152,7 +153,7 @@ public class SimplifyArithmeticRule extends AbstractExpressionRewriteRule {
         }
     }
 
-    private void flattenMultiplyDivide(boolean flag, Expression expr, List<Operand> result) {
+    private static void flattenMultiplyDivide(boolean flag, Expression expr, List<Operand> result) {
         if (TypeUtils.isMultiplyOrDivide(expr)) {
             BinaryArithmetic arithmetic = (BinaryArithmetic) expr;
             flattenMultiplyDivide(flag, arithmetic.left(), result);
@@ -168,13 +169,13 @@ public class SimplifyArithmeticRule extends AbstractExpressionRewriteRule {
         }
     }
 
-    private Expression getSubOrDivide(boolean isAddOrSub, Operand x, Operand y) {
-        return isAddOrSub ? new Subtract(x.expression, y.expression)
+    private static Expression getSubOrDivide(boolean isSubOrDivide, Operand x, Operand y) {
+        return isSubOrDivide ? new Subtract(x.expression, y.expression)
                 : new Divide(x.expression, y.expression);
     }
 
-    private Expression getAddOrMultiply(boolean isAddOrSub, Operand x, Operand y) {
-        return isAddOrSub ? new Add(x.expression, y.expression)
+    private static Expression getAddOrMultiply(boolean isAddOrMultiply, Operand x, Operand y) {
+        return isAddOrMultiply ? new Add(x.expression, y.expression)
                 : new Multiply(x.expression, y.expression);
     }
 
@@ -204,3 +205,4 @@ public class SimplifyArithmeticRule extends AbstractExpressionRewriteRule {
         }
     }
 }
+
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyCastRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyCastRule.java
index 34143043a07..ded0a2f558f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyCastRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyCastRule.java
@@ -17,8 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral;
@@ -37,7 +37,10 @@ import org.apache.doris.nereids.types.DecimalV3Type;
 import org.apache.doris.nereids.types.StringType;
 import org.apache.doris.nereids.types.VarcharType;
 
+import com.google.common.collect.ImmutableList;
+
 import java.math.BigDecimal;
+import java.util.List;
 
 /**
  * Rewrite rule of simplify CAST expression.
@@ -46,17 +49,19 @@ import java.math.BigDecimal;
  * Merge cast like
  * - cast(cast(1 as bigint) as string) -> cast(1 as string).
  */
-public class SimplifyCastRule extends AbstractExpressionRewriteRule {
-
+public class SimplifyCastRule implements ExpressionPatternRuleFactory {
     public static SimplifyCastRule INSTANCE = new SimplifyCastRule();
 
     @Override
-    public Expression visitCast(Cast origin, ExpressionRewriteContext context) {
-        return simplify(origin, context);
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(Cast.class).then(SimplifyCastRule::simplifyCast)
+        );
     }
 
-    private Expression simplify(Cast cast, ExpressionRewriteContext context) {
-        Expression child = rewrite(cast.child(), context);
+    /** simplifyCast */
+    public static Expression simplifyCast(Cast cast) {
+        Expression child = cast.child();
 
         // remove redundant cast
         // CAST(value as type), value is type
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicate.java
index 03958f3d55f..d26b5a53036 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyComparisonPredicate.java
@@ -18,6 +18,8 @@
 package org.apache.doris.nereids.rules.expression.rules;
 
 import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
 import org.apache.doris.nereids.trees.expressions.And;
 import org.apache.doris.nereids.trees.expressions.Cast;
@@ -55,17 +57,18 @@ import org.apache.doris.nereids.types.coercion.DateLikeType;
 import org.apache.doris.nereids.util.TypeCoercionUtils;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 
 import java.math.BigDecimal;
 import java.math.RoundingMode;
+import java.util.List;
 
 /**
  * simplify comparison
  * such as: cast(c1 as DateV2) >= DateV2Literal --> c1 >= DateLiteral
  *          cast(c1 AS double) > 2.0 --> c1 >= 2 (c1 is integer like type)
  */
-public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule {
-
+public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule implements ExpressionPatternRuleFactory {
     public static SimplifyComparisonPredicate INSTANCE = new SimplifyComparisonPredicate();
 
     enum AdjustType {
@@ -74,10 +77,20 @@ public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule {
         NONE
     }
 
+    @Override
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(ComparisonPredicate.class).then(SimplifyComparisonPredicate::simplify)
+        );
+    }
+
     @Override
     public Expression visitComparisonPredicate(ComparisonPredicate cp, ExpressionRewriteContext context) {
-        cp = (ComparisonPredicate) visit(cp, context);
+        return simplify(cp);
+    }
 
+    /** simplify */
+    public static Expression simplify(ComparisonPredicate cp) {
         if (cp.left() instanceof Literal && !(cp.right() instanceof Literal)) {
             cp = cp.commute();
         }
@@ -146,7 +159,7 @@ public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule {
         return comparisonPredicate;
     }
 
-    private Expression processDateLikeTypeCoercion(ComparisonPredicate cp, Expression left, Expression right) {
+    private static Expression processDateLikeTypeCoercion(ComparisonPredicate cp, Expression left, Expression right) {
         if (left instanceof Cast && right instanceof DateLiteral) {
             Cast cast = (Cast) left;
             if (cast.child().getDataType() instanceof DateTimeType) {
@@ -196,7 +209,7 @@ public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule {
         }
     }
 
-    private Expression processFloatLikeTypeCoercion(ComparisonPredicate comparisonPredicate,
+    private static Expression processFloatLikeTypeCoercion(ComparisonPredicate comparisonPredicate,
             Expression left, Expression right) {
         if (left instanceof Cast && left.child(0).getDataType().isIntegerLikeType()
                 && (right instanceof DoubleLiteral || right instanceof FloatLiteral)) {
@@ -209,7 +222,7 @@ public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule {
         }
     }
 
-    private Expression processDecimalV3TypeCoercion(ComparisonPredicate comparisonPredicate,
+    private static Expression processDecimalV3TypeCoercion(ComparisonPredicate comparisonPredicate,
             Expression left, Expression right) {
         if (left instanceof Cast && right instanceof DecimalV3Literal) {
             Cast cast = (Cast) left;
@@ -264,7 +277,7 @@ public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule {
         return comparisonPredicate;
     }
 
-    private Expression processIntegerDecimalLiteralComparison(
+    private static Expression processIntegerDecimalLiteralComparison(
             ComparisonPredicate comparisonPredicate, Expression left, BigDecimal literal) {
         // we only process isIntegerLikeType, which are tinyint, smallint, int, bigint
         if (literal.compareTo(new BigDecimal(Long.MAX_VALUE)) <= 0) {
@@ -306,7 +319,7 @@ public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule {
         return comparisonPredicate;
     }
 
-    private IntegerLikeLiteral convertDecimalToIntegerLikeLiteral(BigDecimal decimal) {
+    private static IntegerLikeLiteral convertDecimalToIntegerLikeLiteral(BigDecimal decimal) {
         Preconditions.checkArgument(
                 decimal.scale() <= 0 && decimal.compareTo(new BigDecimal(Long.MAX_VALUE)) <= 0,
                 "decimal literal must have 0 scale and smaller than Long.MAX_VALUE");
@@ -322,15 +335,15 @@ public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule {
         }
     }
 
-    private Expression migrateToDateTime(DateTimeV2Literal l) {
+    private static Expression migrateToDateTime(DateTimeV2Literal l) {
         return new DateTimeLiteral(l.getYear(), l.getMonth(), l.getDay(), l.getHour(), l.getMinute(), l.getSecond());
     }
 
-    private boolean cannotAdjust(DateTimeLiteral l, ComparisonPredicate cp) {
+    private static boolean cannotAdjust(DateTimeLiteral l, ComparisonPredicate cp) {
         return cp instanceof EqualTo && (l.getHour() != 0 || l.getMinute() != 0 || l.getSecond() != 0);
     }
 
-    private Expression migrateToDateV2(DateTimeLiteral l, AdjustType type) {
+    private static Expression migrateToDateV2(DateTimeLiteral l, AdjustType type) {
         DateV2Literal d = new DateV2Literal(l.getYear(), l.getMonth(), l.getDay());
         if (type == AdjustType.UPPER && (l.getHour() != 0 || l.getMinute() != 0 || l.getSecond() != 0)) {
             d = ((DateV2Literal) d.plusDays(1));
@@ -338,7 +351,7 @@ public class SimplifyComparisonPredicate extends AbstractExpressionRewriteRule {
         return d;
     }
 
-    private Expression migrateToDate(DateV2Literal l) {
+    private static Expression migrateToDate(DateV2Literal l) {
         return new DateLiteral(l.getYear(), l.getMonth(), l.getDay());
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyDecimalV3Comparison.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyDecimalV3Comparison.java
index 6b0426adaad..c3c3c17dd55 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyDecimalV3Comparison.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyDecimalV3Comparison.java
@@ -17,8 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
 import org.apache.doris.nereids.trees.expressions.Expression;
@@ -26,8 +26,10 @@ import org.apache.doris.nereids.trees.expressions.literal.DecimalV3Literal;
 import org.apache.doris.nereids.types.DecimalV3Type;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 
 import java.math.BigDecimal;
+import java.util.List;
 
 /**
  * if we have a column with decimalv3 type and set enable_decimal_conversion = false.
@@ -37,14 +39,20 @@ import java.math.BigDecimal;
  * and the col1 need to convert to decimalv3(27, 9) to match the precision of right hand
  * this rule simplify it from cast(col1 as decimalv3(27, 9)) > 0.6 to col1 > 0.6
  */
-public class SimplifyDecimalV3Comparison extends AbstractExpressionRewriteRule {
-
+public class SimplifyDecimalV3Comparison implements ExpressionPatternRuleFactory {
     public static SimplifyDecimalV3Comparison INSTANCE = new SimplifyDecimalV3Comparison();
 
     @Override
-    public Expression visitComparisonPredicate(ComparisonPredicate cp, ExpressionRewriteContext context) {
-        Expression left = rewrite(cp.left(), context);
-        Expression right = rewrite(cp.right(), context);
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(ComparisonPredicate.class).then(SimplifyDecimalV3Comparison::simplify)
+        );
+    }
+
+    /** simplify */
+    public static Expression simplify(ComparisonPredicate cp) {
+        Expression left = cp.left();
+        Expression right = cp.right();
 
         if (left.getDataType() instanceof DecimalV3Type
                 && left instanceof Cast
@@ -60,7 +68,7 @@ public class SimplifyDecimalV3Comparison extends AbstractExpressionRewriteRule {
         }
     }
 
-    private Expression doProcess(ComparisonPredicate cp, Cast left, DecimalV3Literal right) {
+    private static Expression doProcess(ComparisonPredicate cp, Cast left, DecimalV3Literal right) {
         BigDecimal trailingZerosValue = right.getValue().stripTrailingZeros();
         int scale = org.apache.doris.analysis.DecimalLiteral.getBigDecimalScale(trailingZerosValue);
         int precision = org.apache.doris.analysis.DecimalLiteral.getBigDecimalPrecision(trailingZerosValue);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyInPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyInPredicate.java
index 3e194a4edde..bf1b194a6ac 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyInPredicate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyInPredicate.java
@@ -17,8 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.InPredicate;
@@ -33,12 +33,18 @@ import java.util.List;
 /**
  * SimplifyInPredicate
  */
-public class SimplifyInPredicate extends AbstractExpressionRewriteRule {
-
+public class SimplifyInPredicate implements ExpressionPatternRuleFactory {
     public static final SimplifyInPredicate INSTANCE = new SimplifyInPredicate();
 
     @Override
-    public Expression visitInPredicate(InPredicate expr, ExpressionRewriteContext context) {
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(InPredicate.class).then(SimplifyInPredicate::simplify)
+        );
+    }
+
+    /** simplify */
+    public static Expression simplify(InPredicate expr) {
         if (expr.children().size() > 1) {
             if (expr.getCompareExpr() instanceof Cast) {
                 Cast cast = (Cast) expr.getCompareExpr();
@@ -58,7 +64,7 @@ public class SimplifyInPredicate extends AbstractExpressionRewriteRule {
                     DateTimeV2Type compareType = (DateTimeV2Type) cast.child().getDataType();
                     if (literals.stream().allMatch(literal -> literal instanceof DateTimeV2Literal
                             && canLosslessConvertToLowScaleLiteral(
-                                    (DateTimeV2Literal) literal, compareType.getScale()))) {
+                            (DateTimeV2Literal) literal, compareType.getScale()))) {
                         ImmutableList.Builder<Expression> children = ImmutableList.builder();
                         children.add(cast.child());
                         literals.forEach(l -> children.add(new DateTimeV2Literal(compareType,
@@ -86,7 +92,7 @@ public class SimplifyInPredicate extends AbstractExpressionRewriteRule {
                 | literal.getMicroSecond()) == 0L;
     }
 
-    private DateV2Literal convertToDateV2Literal(DateTimeV2Literal literal) {
+    private static DateV2Literal convertToDateV2Literal(DateTimeV2Literal literal) {
         return new DateV2Literal(literal.getYear(), literal.getMonth(), literal.getDay());
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyNotExprRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyNotExprRule.java
index 7268d6e8328..484d68f0d73 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyNotExprRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyNotExprRule.java
@@ -17,8 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
 import org.apache.doris.nereids.trees.expressions.CompoundPredicate;
 import org.apache.doris.nereids.trees.expressions.Expression;
@@ -28,6 +28,10 @@ import org.apache.doris.nereids.trees.expressions.LessThan;
 import org.apache.doris.nereids.trees.expressions.LessThanEqual;
 import org.apache.doris.nereids.trees.expressions.Not;
 
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
 /**
  * Rewrite rule of NOT expression.
  * For example:
@@ -42,12 +46,19 @@ import org.apache.doris.nereids.trees.expressions.Not;
  * not and(a >= b, a <= c) -> or(a < b, a > c)
  * not or(a >= b, a <= c) -> and(a < b, a > c)
  */
-public class SimplifyNotExprRule extends AbstractExpressionRewriteRule {
+public class SimplifyNotExprRule implements ExpressionPatternRuleFactory {
 
     public static SimplifyNotExprRule INSTANCE = new SimplifyNotExprRule();
 
     @Override
-    public Expression visitNot(Not not, ExpressionRewriteContext context) {
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(Not.class).then(SimplifyNotExprRule::simplify)
+        );
+    }
+
+    /** simplifyNot */
+    public static Expression simplify(Not not) {
         Expression child = not.child();
         if (child instanceof ComparisonPredicate) {
             ComparisonPredicate cp = (ComparisonPredicate) not.child();
@@ -55,23 +66,22 @@ public class SimplifyNotExprRule extends AbstractExpressionRewriteRule {
             Expression right = cp.right();
 
             if (child instanceof GreaterThan) {
-                return new LessThanEqual(left, right).accept(this, context);
+                return new LessThanEqual(left, right);
             } else if (child instanceof GreaterThanEqual) {
-                return new LessThan(left, right).accept(this, context);
+                return new LessThan(left, right);
             } else if (child instanceof LessThan) {
-                return new GreaterThanEqual(left, right).accept(this, context);
+                return new GreaterThanEqual(left, right);
             } else if (child instanceof LessThanEqual) {
-                return new GreaterThan(left, right).accept(this, context);
+                return new GreaterThan(left, right);
             }
         } else if (child instanceof CompoundPredicate) {
             CompoundPredicate cp = (CompoundPredicate) child;
             Not left = new Not(cp.left());
             Not right = new Not(cp.right());
-            return cp.flip(left, right).accept(this, context);
+            return cp.flip(left, right);
         } else if (child instanceof Not) {
-            return child.child(0).accept(this, context);
+            return child.child(0);
         }
-
-        return super.visitNot(not, context);
+        return not;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java
index 4dbfdb2f35a..98d752facb4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifyRange.java
@@ -17,9 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
-import org.apache.doris.nereids.rules.expression.ExpressionRuleExecutor;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.And;
 import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
 import org.apache.doris.nereids.trees.expressions.CompoundPredicate;
@@ -41,15 +40,17 @@ import org.apache.doris.nereids.util.ExpressionUtils;
 import com.google.common.collect.BoundType;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
 import com.google.common.collect.Range;
 import com.google.common.collect.Sets;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.function.BinaryOperator;
@@ -74,18 +75,21 @@ import java.util.stream.Collectors;
  * 2. for `Or` expression (similar to `And`).
  * todo: support a > 10 and (a < 10 or a > 20 ) => a > 20
  */
-public class SimplifyRange extends AbstractExpressionRewriteRule {
-
+public class SimplifyRange implements ExpressionPatternRuleFactory {
     public static final SimplifyRange INSTANCE = new SimplifyRange();
 
     @Override
-    public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) {
-        if (expr instanceof CompoundPredicate) {
-            ValueDesc valueDesc = expr.accept(new RangeInference(), null);
-            Expression simplifiedExpr = valueDesc.toExpression();
-            return simplifiedExpr == null ? valueDesc.expr : simplifiedExpr;
-        }
-        return expr;
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesTopType(CompoundPredicate.class).then(SimplifyRange::rewrite)
+        );
+    }
+
+    /** rewrite */
+    public static Expression rewrite(CompoundPredicate expr) {
+        ValueDesc valueDesc = expr.accept(new RangeInference(), null);
+        Expression simplifiedExpr = valueDesc.toExpression();
+        return simplifiedExpr == null ? valueDesc.expr : simplifiedExpr;
     }
 
     private static class RangeInference extends ExpressionVisitor<ValueDesc, Void> {
@@ -96,21 +100,20 @@ public class SimplifyRange extends AbstractExpressionRewriteRule {
         }
 
         private ValueDesc buildRange(ComparisonPredicate predicate) {
-            Expression rewrite = ExpressionRuleExecutor.normalize(predicate);
-            Expression right = rewrite.child(1);
+            Expression right = predicate.child(1);
             if (right.isNullLiteral()) {
                 // it's safe to return empty value if >, >=, <, <= and = with null
                 if ((predicate instanceof GreaterThan || predicate instanceof GreaterThanEqual
                         || predicate instanceof LessThan || predicate instanceof LessThanEqual
                         || predicate instanceof EqualTo)) {
-                    return new EmptyValue(rewrite.child(0), rewrite);
+                    return new EmptyValue(predicate.child(0), predicate);
                 } else {
                     return new UnknownValue(predicate);
                 }
             }
             // only handle `NumericType` and `DateLikeType`
             if (right.isLiteral() && (right.getDataType().isNumericType() || right.getDataType().isDateLikeType())) {
-                return ValueDesc.range((ComparisonPredicate) rewrite);
+                return ValueDesc.range(predicate);
             }
             return new UnknownValue(predicate);
         }
@@ -164,18 +167,23 @@ public class SimplifyRange extends AbstractExpressionRewriteRule {
         private ValueDesc simplify(Expression originExpr, List<Expression> predicates,
                 BinaryOperator<ValueDesc> op, BinaryOperator<Expression> exprOp) {
 
-            Map<Expression, List<ValueDesc>> groupByReference = predicates.stream()
-                    .map(predicate -> predicate.accept(this, null))
-                    .collect(Collectors.groupingBy(p -> p.reference, LinkedHashMap::new, Collectors.toList()));
+            Multimap<Expression, ValueDesc> groupByReference
+                    = Multimaps.newListMultimap(new LinkedHashMap<>(), ArrayList::new);
+            for (Expression predicate : predicates) {
+                ValueDesc valueDesc = predicate.accept(this, null);
+                List<ValueDesc> valueDescs = (List<ValueDesc>) groupByReference.get(valueDesc.reference);
+                valueDescs.add(valueDesc);
+            }
 
             List<ValueDesc> valuePerRefs = Lists.newArrayList();
-            for (Entry<Expression, List<ValueDesc>> referenceValues : groupByReference.entrySet()) {
-                List<ValueDesc> valuePerReference = referenceValues.getValue();
+            for (Entry<Expression, Collection<ValueDesc>> referenceValues : groupByReference.asMap().entrySet()) {
+                List<ValueDesc> valuePerReference = (List) referenceValues.getValue();
 
                 // merge per reference
-                ValueDesc simplifiedValue = valuePerReference.stream()
-                        .reduce(op)
-                        .get();
+                ValueDesc simplifiedValue = valuePerReference.get(0);
+                for (int i = 1; i < valuePerReference.size(); i++) {
+                    simplifiedValue = op.apply(simplifiedValue, valuePerReference.get(i));
+                }
 
                 valuePerRefs.add(simplifiedValue);
             }
@@ -245,6 +253,7 @@ public class SimplifyRange extends AbstractExpressionRewriteRule {
         }
 
         public static ValueDesc discrete(InPredicate in) {
+            // Set<Literal> literals = (Set) Utils.fastToImmutableSet(in.getOptions());
             Set<Literal> literals = in.getOptions().stream().map(Literal.class::cast).collect(Collectors.toSet());
             return new DiscreteValue(in.getCompareExpr(), in, literals);
         }
@@ -427,7 +436,9 @@ public class SimplifyRange extends AbstractExpressionRewriteRule {
             // They are same processes, so must change synchronously.
             if (values.size() == 1) {
                 return new EqualTo(reference, values.iterator().next());
-            } else if (values.size() <= OrToIn.REWRITE_OR_TO_IN_PREDICATE_THRESHOLD) {
+
+                // this condition should as same as OrToIn, or else meet dead loop
+            } else if (values.size() < OrToIn.REWRITE_OR_TO_IN_PREDICATE_THRESHOLD) {
                 Iterator<Literal> iterator = values.iterator();
                 return new Or(new EqualTo(reference, iterator.next()), new EqualTo(reference, iterator.next()));
             } else {
@@ -478,10 +489,12 @@ public class SimplifyRange extends AbstractExpressionRewriteRule {
             if (sourceValues.isEmpty()) {
                 return expr;
             }
-            return sourceValues.stream()
-                    .map(ValueDesc::toExpression)
-                    .reduce(mergeExprOp)
-                    .get();
+
+            Expression result = sourceValues.get(0).toExpression();
+            for (int i = 1; i < sourceValues.size(); i++) {
+                result = mergeExprOp.apply(result, sourceValues.get(i).toExpression());
+            }
+            return result;
         }
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SupportJavaDateFormatter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SupportJavaDateFormatter.java
index 17f4b7d239a..27b929a2b9f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SupportJavaDateFormatter.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SupportJavaDateFormatter.java
@@ -17,8 +17,8 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.DateFormat;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.FromUnixtime;
@@ -26,54 +26,46 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.UnixTimestamp
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.util.List;
 
-/** SupportJavaDateFormatter */
-public class SupportJavaDateFormatter extends AbstractExpressionRewriteRule {
+/** SupportJavaDateFormatter2 */
+public class SupportJavaDateFormatter implements ExpressionPatternRuleFactory {
     public static final SupportJavaDateFormatter INSTANCE = new SupportJavaDateFormatter();
 
     @Override
-    public Expression visitDateFormat(DateFormat dateFormat, ExpressionRewriteContext context) {
-        Expression expr = super.visitDateFormat(dateFormat, context);
-        if (!(expr instanceof DateFormat)) {
-            return expr;
-        }
-        dateFormat = (DateFormat) expr;
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesType(DateFormat.class).then(SupportJavaDateFormatter::rewriteDateFormat),
+                matchesType(FromUnixtime.class).then(SupportJavaDateFormatter::rewriteFromUnixtime),
+                matchesType(UnixTimestamp.class).then(SupportJavaDateFormatter::rewriteUnixTimestamp)
+        );
+    }
+
+    public static Expression rewriteDateFormat(DateFormat dateFormat) {
         if (dateFormat.arity() > 1) {
             return translateJavaFormatter(dateFormat, 1);
         }
         return dateFormat;
     }
 
-    @Override
-    public Expression visitFromUnixtime(FromUnixtime fromUnixtime, ExpressionRewriteContext context) {
-        Expression expr = super.visitFromUnixtime(fromUnixtime, context);
-        if (!(expr instanceof FromUnixtime)) {
-            return expr;
-        }
-        fromUnixtime = (FromUnixtime) expr;
+    public static Expression rewriteFromUnixtime(FromUnixtime fromUnixtime) {
         if (fromUnixtime.arity() > 1) {
             return translateJavaFormatter(fromUnixtime, 1);
         }
         return fromUnixtime;
     }
 
-    @Override
-    public Expression visitUnixTimestamp(UnixTimestamp unixTimestamp, ExpressionRewriteContext context) {
-        Expression expr = super.visitUnixTimestamp(unixTimestamp, context);
-        if (!(expr instanceof UnixTimestamp)) {
-            return expr;
-        }
-        unixTimestamp = (UnixTimestamp) expr;
+    public static Expression rewriteUnixTimestamp(UnixTimestamp unixTimestamp) {
         if (unixTimestamp.arity() > 1) {
             return translateJavaFormatter(unixTimestamp, 1);
         }
         return unixTimestamp;
     }
 
-    private Expression translateJavaFormatter(Expression function, int formatterIndex) {
+    private static Expression translateJavaFormatter(Expression function, int formatterIndex) {
         Expression formatterExpr = function.getArgument(formatterIndex);
         Expression newFormatterExpr = translateJavaFormatter(formatterExpr);
         if (newFormatterExpr != formatterExpr) {
@@ -84,7 +76,7 @@ public class SupportJavaDateFormatter extends AbstractExpressionRewriteRule {
         return function;
     }
 
-    private Expression translateJavaFormatter(Expression formatterExpr) {
+    private static Expression translateJavaFormatter(Expression formatterExpr) {
         if (formatterExpr.isLiteral() && formatterExpr.getDataType().isStringLikeType()) {
             Literal literal = (Literal) formatterExpr;
             String originFormatter = literal.getStringValue();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TopnToMax.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TopnToMax.java
index 30e76cfe226..318cb6ec603 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TopnToMax.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TopnToMax.java
@@ -17,39 +17,38 @@
 
 package org.apache.doris.nereids.rules.expression.rules;
 
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
-import org.apache.doris.nereids.rules.expression.ExpressionRewriteRule;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
 import org.apache.doris.nereids.trees.expressions.Expression;
-import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Max;
 import org.apache.doris.nereids.trees.expressions.functions.agg.TopN;
 import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral;
-import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
 
 /**
  * Convert topn(x, 1) to max(x)
  */
-public class TopnToMax extends DefaultExpressionRewriter<ExpressionRewriteContext> implements
-        ExpressionRewriteRule<ExpressionRewriteContext> {
+public class TopnToMax implements ExpressionPatternRuleFactory {
 
     public static final TopnToMax INSTANCE = new TopnToMax();
 
     @Override
-    public Expression rewrite(Expression expr, ExpressionRewriteContext ctx) {
-        return expr.accept(this, null);
+    public List<ExpressionPatternMatcher<? extends Expression>> buildRules() {
+        return ImmutableList.of(
+                matchesTopType(TopN.class).then(TopnToMax::rewrite)
+        );
     }
 
-    @Override
-    public Expression visitAggregateFunction(AggregateFunction aggregateFunction, ExpressionRewriteContext context) {
-        if (!(aggregateFunction instanceof TopN)) {
-            return aggregateFunction;
-        }
-        TopN topN = (TopN) aggregateFunction;
+    /** rewrite */
+    public static Expression rewrite(TopN topN) {
         if (topN.arity() == 2 && topN.child(1) instanceof IntegerLikeLiteral
                 && ((IntegerLikeLiteral) topN.child(1)).getIntValue() == 1) {
             return new Max(topN.child(0));
         } else {
-            return aggregateFunction;
+            return topN;
         }
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TryEliminateUninterestedPredicates.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TryEliminateUninterestedPredicates.java
index 3faf56f0f38..ce23219bcc9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TryEliminateUninterestedPredicates.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/TryEliminateUninterestedPredicates.java
@@ -22,6 +22,7 @@ import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
 import org.apache.doris.nereids.rules.expression.rules.TryEliminateUninterestedPredicates.Context;
 import org.apache.doris.nereids.trees.expressions.And;
 import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.Not;
 import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral;
 import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter;
@@ -51,10 +52,17 @@ public class TryEliminateUninterestedPredicates extends DefaultExpressionRewrite
         this.expressionRewriteContext = new ExpressionRewriteContext(cascadesContext);
     }
 
+    /** rewrite */
     public static Expression rewrite(Expression expression, Set<Slot> interestedSlots,
             CascadesContext cascadesContext) {
         // before eliminate uninterested predicate, we must push down `Not` under CompoundPredicate
-        expression = expression.accept(new SimplifyNotExprRule(), null);
+        expression = expression.rewriteUp(expr -> {
+            if (expr instanceof Not) {
+                return SimplifyNotExprRule.simplify((Not) expr);
+            } else {
+                return expr;
+            }
+        });
         TryEliminateUninterestedPredicates rewriter = new TryEliminateUninterestedPredicates(
                 interestedSlots, cascadesContext);
         return expression.accept(rewriter, new Context());
@@ -89,7 +97,7 @@ public class TryEliminateUninterestedPredicates extends DefaultExpressionRewrite
                 // -> ((interested slot a) and true) or true
                 // -> (interested slot a) or true
                 // -> true
-                expr = expr.accept(FoldConstantRuleOnFE.INSTANCE, expressionRewriteContext);
+                expr = FoldConstantRuleOnFE.evaluate(expr, expressionRewriteContext);
             }
         } else {
             //    ((uninterested slot b > 0) + 1) > 1
@@ -122,7 +130,7 @@ public class TryEliminateUninterestedPredicates extends DefaultExpressionRewrite
         if (rightContext.childrenContainsNonInterestedSlots) {
             newRight = BooleanLiteral.TRUE;
         }
-        Expression expr = new And(newLeft, newRight).accept(FoldConstantRuleOnFE.INSTANCE, expressionRewriteContext);
+        Expression expr = FoldConstantRuleOnFE.evaluate(new And(newLeft, newRight), expressionRewriteContext);
         parentContext.childrenContainsInterestedSlots =
                 rightContext.childrenContainsInterestedSlots || leftContext.childrenContainsInterestedSlots;
         return expr;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java
index 10b21d0b979..61aac4d2407 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java
@@ -1611,7 +1611,7 @@ public class AggregateStrategies implements ImplementationRuleFactory {
     }
 
     private boolean containsCountDistinctMultiExpr(LogicalAggregate<? extends Plan> aggregate) {
-        return ExpressionUtils.anyMatch(aggregate.getOutputExpressions(), expr ->
+        return ExpressionUtils.deapAnyMatch(aggregate.getOutputExpressions(), expr ->
                 expr instanceof Count && ((Count) expr).isDistinct() && expr.arity() > 1);
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustConjunctsReturnType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustConjunctsReturnType.java
index fcf3e82737b..ebc67e7c515 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustConjunctsReturnType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustConjunctsReturnType.java
@@ -27,6 +27,8 @@ import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter;
 import org.apache.doris.nereids.types.BooleanType;
 import org.apache.doris.nereids.util.TypeCoercionUtils;
 
+import com.google.common.collect.ImmutableSet;
+
 import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
@@ -51,7 +53,7 @@ public class AdjustConjunctsReturnType extends DefaultPlanRewriter<Void> impleme
         filter = (LogicalFilter<? extends Plan>) super.visit(filter, context);
         Set<Expression> conjuncts = filter.getConjuncts().stream()
                 .map(expr -> TypeCoercionUtils.castIfNotSameType(expr, BooleanType.INSTANCE))
-                .collect(Collectors.toSet());
+                .collect(ImmutableSet.toImmutableSet());
         return filter.withConjuncts(conjuncts);
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java
index f30d55ad0fc..a608448e023 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java
@@ -268,11 +268,19 @@ public class AdjustNullable extends DefaultPlanRewriter<Map<ExprId, Slot>> imple
     }
 
     private <T extends Expression> List<T> updateExpressions(List<T> inputs, Map<ExprId, Slot> replaceMap) {
-        return inputs.stream().map(i -> updateExpression(i, replaceMap)).collect(ImmutableList.toImmutableList());
+        ImmutableList.Builder<T> result = ImmutableList.builderWithExpectedSize(inputs.size());
+        for (T input : inputs) {
+            result.add(updateExpression(input, replaceMap));
+        }
+        return result.build();
     }
 
     private <T extends Expression> Set<T> updateExpressions(Set<T> inputs, Map<ExprId, Slot> replaceMap) {
-        return inputs.stream().map(i -> updateExpression(i, replaceMap)).collect(ImmutableSet.toImmutableSet());
+        ImmutableSet.Builder<T> result = ImmutableSet.builderWithExpectedSize(inputs.size());
+        for (T input : inputs) {
+            result.add(updateExpression(input, replaceMap));
+        }
+        return result.build();
     }
 
     private Map<ExprId, Slot> collectChildrenOutputMap(LogicalPlan plan) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckMatchExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckMatchExpression.java
index 907d34c07c0..8c73991f363 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckMatchExpression.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckMatchExpression.java
@@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.SlotReference;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.util.ExpressionUtils;
 
 import java.util.List;
 
@@ -38,7 +39,7 @@ public class CheckMatchExpression extends OneRewriteRuleFactory {
     @Override
     public Rule build() {
         return logicalFilter(logicalOlapScan())
-                .when(filter -> containsMatchExpression(filter.getExpressions()))
+                .when(filter -> ExpressionUtils.containsType(filter.getExpressions(), Match.class))
                 .then(this::checkChildren)
                 .toRule(RuleType.CHECK_MATCH_EXPRESSION);
     }
@@ -60,8 +61,4 @@ public class CheckMatchExpression extends OneRewriteRuleFactory {
         }
         return filter;
     }
-
-    private boolean containsMatchExpression(List<Expression> expressions) {
-        return expressions.stream().anyMatch(expr -> expr.anyMatch(Match.class::isInstance));
-    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java
index b4d7b800513..70a5c593ee3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java
@@ -22,7 +22,6 @@ import org.apache.doris.common.UserException;
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.jobs.JobContext;
 import org.apache.doris.nereids.rules.analysis.UserAuthentication;
-import org.apache.doris.nereids.trees.expressions.NamedExpression;
 import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation;
@@ -30,9 +29,12 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalRelation;
 import org.apache.doris.nereids.trees.plans.logical.LogicalView;
 import org.apache.doris.qe.ConnectContext;
 
+import com.google.common.collect.Sets;
+
+import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 /** CheckPrivileges */
 public class CheckPrivileges extends ColumnPruning {
@@ -65,15 +67,20 @@ public class CheckPrivileges extends ColumnPruning {
     }
 
     private Set<String> computeUsedColumns(Plan plan, Set<Slot> requiredSlots) {
-        Map<Integer, Slot> idToSlot = plan.getOutputSet()
-                .stream()
-                .collect(Collectors.toMap(slot -> slot.getExprId().asInt(), slot -> slot));
-        return requiredSlots
-                .stream()
-                .map(slot -> idToSlot.get(slot.getExprId().asInt()))
-                .filter(slot -> slot != null)
-                .map(NamedExpression::getName)
-                .collect(Collectors.toSet());
+        List<Slot> outputs = plan.getOutput();
+        Map<Integer, Slot> idToSlot = new LinkedHashMap<>(outputs.size());
+        for (Slot output : outputs) {
+            idToSlot.putIfAbsent(output.getExprId().asInt(), output);
+        }
+
+        Set<String> usedColumns = Sets.newLinkedHashSetWithExpectedSize(requiredSlots.size());
+        for (Slot requiredSlot : requiredSlots) {
+            Slot slot = idToSlot.get(requiredSlot.getExprId().asInt());
+            if (slot != null) {
+                usedColumns.add(slot.getName());
+            }
+        }
+        return usedColumns;
     }
 
     private void checkColumnPrivileges(TableIf table, Set<String> usedColumns) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java
index f33f1658c32..e36c0f5172a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java
@@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.NamedExpression;
 import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.expressions.SlotReference;
+import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.algebra.Aggregate;
 import org.apache.doris.nereids.trees.plans.algebra.SetOperation.Qualifier;
@@ -39,18 +40,17 @@ import org.apache.doris.nereids.trees.plans.logical.OutputPrunable;
 import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter;
 import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter;
 import org.apache.doris.nereids.util.ExpressionUtils;
+import org.apache.doris.nereids.util.Utils;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
 import java.util.function.Function;
-import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
 /**
@@ -97,13 +97,11 @@ public class ColumnPruning extends DefaultPlanRewriter<PruneContext> implements
             for (Plan child : plan.children()) {
                 child.accept(this, jobContext);
             }
-            plan.getExpressions().stream().filter(
-                    expression -> !(expression instanceof SlotReference)
-            ).forEach(
-                    expression -> {
-                        keys.addAll(expression.getInputSlots());
-                    }
-            );
+            for (Expression expression : plan.getExpressions()) {
+                if (!(expression instanceof SlotReference)) {
+                    keys.addAll(expression.getInputSlots());
+                }
+            }
             return plan;
         }
     }
@@ -212,39 +210,42 @@ public class ColumnPruning extends DefaultPlanRewriter<PruneContext> implements
     }
 
     private Plan skipPruneThisAndFirstLevelChildren(Plan plan) {
-        Set<Slot> requireAllOutputOfChildren = plan.children()
-                .stream()
-                .flatMap(child -> child.getOutputSet().stream())
-                .collect(Collectors.toSet());
-        return pruneChildren(plan, requireAllOutputOfChildren);
+        ImmutableSet.Builder<Slot> requireAllOutputOfChildren = ImmutableSet.builder();
+        for (Plan child : plan.children()) {
+            requireAllOutputOfChildren.addAll(child.getOutput());
+        }
+        return pruneChildren(plan, requireAllOutputOfChildren.build());
     }
 
-    private static Aggregate fillUpGroupByAndOutput(Aggregate prunedOutputAgg) {
+    private static Aggregate<Plan> fillUpGroupByAndOutput(Aggregate<Plan> prunedOutputAgg) {
         List<Expression> groupBy = prunedOutputAgg.getGroupByExpressions();
         List<NamedExpression> output = prunedOutputAgg.getOutputExpressions();
 
         if (!(prunedOutputAgg instanceof LogicalAggregate)) {
             return prunedOutputAgg;
         }
-        // add back group by keys which eliminated by rule ELIMINATE_GROUP_BY_KEY
-        // if related output expressions are not in pruned output list.
-        List<NamedExpression> remainedOutputExprs = Lists.newArrayList(output);
-        remainedOutputExprs.removeAll(groupBy);
 
-        List<NamedExpression> newOutputList = Lists.newArrayList();
-        newOutputList.addAll((List) groupBy);
-        newOutputList.addAll(remainedOutputExprs);
+        ImmutableList.Builder<NamedExpression> newOutputListBuilder
+                = ImmutableList.builderWithExpectedSize(output.size());
+        newOutputListBuilder.addAll((List) groupBy);
+        for (NamedExpression ne : output) {
+            if (!groupBy.contains(ne)) {
+                newOutputListBuilder.add(ne);
+            }
+        }
 
-        if (!(prunedOutputAgg instanceof LogicalAggregate)) {
-            return prunedOutputAgg.withAggOutput(newOutputList);
-        } else {
-            List<Expression> newGroupByExprList = newOutputList.stream().filter(e ->
-                    !(prunedOutputAgg.getAggregateFunctions().contains(e)
-                            || e instanceof Alias && prunedOutputAgg.getAggregateFunctions()
-                                .contains(((Alias) e).child()))
-            ).collect(Collectors.toList());
-            return ((LogicalAggregate) prunedOutputAgg).withGroupByAndOutput(newGroupByExprList, newOutputList);
+        List<NamedExpression> newOutputList = newOutputListBuilder.build();
+        Set<AggregateFunction> aggregateFunctions = prunedOutputAgg.getAggregateFunctions();
+        ImmutableList.Builder<Expression> newGroupByExprList
+                = ImmutableList.builderWithExpectedSize(newOutputList.size());
+        for (NamedExpression e : newOutputList) {
+            if (!(aggregateFunctions.contains(e)
+                    || (e instanceof Alias && aggregateFunctions.contains(e.child(0))))) {
+                newGroupByExprList.add(e);
+            }
         }
+        return ((LogicalAggregate<Plan>) prunedOutputAgg).withGroupByAndOutput(
+                newGroupByExprList.build(), newOutputList);
     }
 
     /** prune output */
@@ -253,9 +254,8 @@ public class ColumnPruning extends DefaultPlanRewriter<PruneContext> implements
         if (originOutput.isEmpty()) {
             return plan;
         }
-        List<NamedExpression> prunedOutputs = originOutput.stream()
-                .filter(output -> context.requiredSlots.contains(output.toSlot()))
-                .collect(ImmutableList.toImmutableList());
+        List<NamedExpression> prunedOutputs =
+                Utils.filterImmutableList(originOutput, output -> context.requiredSlots.contains(output.toSlot()));
 
         if (prunedOutputs.isEmpty()) {
             List<NamedExpression> candidates = Lists.newArrayList(originOutput);
@@ -281,7 +281,6 @@ public class ColumnPruning extends DefaultPlanRewriter<PruneContext> implements
         }
         List<NamedExpression> prunedOutputs = Lists.newArrayList();
         List<List<NamedExpression>> constantExprsList = union.getConstantExprsList();
-        List<List<NamedExpression>> prunedConstantExprsList = Lists.newArrayList();
         List<Integer> extractColumnIndex = Lists.newArrayList();
         for (int i = 0; i < originOutput.size(); i++) {
             NamedExpression output = originOutput.get(i);
@@ -291,12 +290,14 @@ public class ColumnPruning extends DefaultPlanRewriter<PruneContext> implements
             }
         }
         int len = extractColumnIndex.size();
+        ImmutableList.Builder<List<NamedExpression>> prunedConstantExprsList
+                = ImmutableList.builderWithExpectedSize(constantExprsList.size());
         for (List<NamedExpression> row : constantExprsList) {
-            ArrayList<NamedExpression> newRow = new ArrayList<>(len);
+            ImmutableList.Builder<NamedExpression> newRow = ImmutableList.builderWithExpectedSize(len);
             for (int idx : extractColumnIndex) {
                 newRow.add(row.get(idx));
             }
-            prunedConstantExprsList.add(newRow);
+            prunedConstantExprsList.add(newRow.build());
         }
 
         if (prunedOutputs.isEmpty()) {
@@ -312,7 +313,7 @@ public class ColumnPruning extends DefaultPlanRewriter<PruneContext> implements
         if (prunedOutputs.equals(originOutput)) {
             return union;
         } else {
-            return union.withNewOutputsAndConstExprsList(prunedOutputs, prunedConstantExprsList);
+            return union.withNewOutputsAndConstExprsList(prunedOutputs, prunedConstantExprsList.build());
         }
     }
 
@@ -329,24 +330,31 @@ public class ColumnPruning extends DefaultPlanRewriter<PruneContext> implements
         Set<Slot> currentUsedSlots = plan.getInputSlots();
         Set<Slot> childrenRequiredSlots = parentRequiredSlots.isEmpty()
                 ? currentUsedSlots
-                : ImmutableSet.<Slot>builder()
+                : ImmutableSet.<Slot>builderWithExpectedSize(parentRequiredSlots.size() + currentUsedSlots.size())
                         .addAll(parentRequiredSlots)
                         .addAll(currentUsedSlots)
                         .build();
 
-        List<Plan> newChildren = new ArrayList<>();
+        ImmutableList.Builder<Plan> newChildren = ImmutableList.builderWithExpectedSize(plan.arity());
         boolean hasNewChildren = false;
         for (Plan child : plan.children()) {
-            Set<Slot> childOutputSet = child.getOutputSet();
-            Set<Slot> childRequiredSlots = childOutputSet.stream()
-                    .filter(childrenRequiredSlots::contains).collect(Collectors.toSet());
+            Set<Slot> childRequiredSlots;
+            List<Slot> childOutputs = child.getOutput();
+            ImmutableSet.Builder<Slot> childRequiredSlotBuilder
+                    = ImmutableSet.builderWithExpectedSize(childOutputs.size());
+            for (Slot childOutput : childOutputs) {
+                if (childrenRequiredSlots.contains(childOutput)) {
+                    childRequiredSlotBuilder.add(childOutput);
+                }
+            }
+            childRequiredSlots = childRequiredSlotBuilder.build();
             Plan prunedChild = doPruneChild(plan, child, childRequiredSlots);
             if (prunedChild != child) {
                 hasNewChildren = true;
             }
             newChildren.add(prunedChild);
         }
-        return hasNewChildren ? (P) plan.withChildren(newChildren) : plan;
+        return hasNewChildren ? (P) plan.withChildren(newChildren.build()) : plan;
     }
 
     private Plan doPruneChild(Plan plan, Plan child, Set<Slot> childRequiredSlots) {
@@ -358,7 +366,7 @@ public class ColumnPruning extends DefaultPlanRewriter<PruneContext> implements
 
         // the case 2 in the class comment, prune child's output failed
         if (!isProject && !Sets.difference(prunedChild.getOutputSet(), childRequiredSlots).isEmpty()) {
-            prunedChild = new LogicalProject<>(ImmutableList.copyOf(childRequiredSlots), prunedChild);
+            prunedChild = new LogicalProject<>(Utils.fastToImmutableList(childRequiredSlots), prunedChild);
         }
         return prunedChild;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CountDistinctRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CountDistinctRewrite.java
index f2ccf55ac50..3d106078a03 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CountDistinctRewrite.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CountDistinctRewrite.java
@@ -24,9 +24,12 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression;
 import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnionCount;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Count;
 import org.apache.doris.nereids.trees.expressions.functions.agg.HllUnionAgg;
-import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.types.DataType;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableList.Builder;
 
 import java.util.List;
 
@@ -38,35 +41,42 @@ import java.util.List;
 public class CountDistinctRewrite extends OneRewriteRuleFactory {
     @Override
     public Rule build() {
-        return logicalAggregate().then(agg -> {
-            List<NamedExpression> output = agg.getOutputExpressions()
-                    .stream()
-                    .map(CountDistinctRewriter::rewrite)
-                    .map(NamedExpression.class::cast)
-                    .collect(ImmutableList.toImmutableList());
-            return agg.withAggOutput(output);
+        return logicalAggregate().when(CountDistinctRewrite::containsCountObject).then(agg -> {
+            List<NamedExpression> outputExpressions = agg.getOutputExpressions();
+            Builder<NamedExpression> newOutputs
+                    = ImmutableList.builderWithExpectedSize(outputExpressions.size());
+            for (NamedExpression outputExpression : outputExpressions) {
+                NamedExpression newOutput = (NamedExpression) outputExpression.rewriteUp(expr -> {
+                    if (expr instanceof Count && ((Count) expr).isDistinct() && expr.arity() == 1) {
+                        Expression child = expr.child(0);
+                        if (child.getDataType().isBitmapType()) {
+                            return new BitmapUnionCount(child);
+                        }
+                        if (child.getDataType().isHllType()) {
+                            return new HllUnionAgg(child);
+                        }
+                    }
+                    return expr;
+                });
+                newOutputs.add(newOutput);
+            }
+            return agg.withAggOutput(newOutputs.build());
         }).toRule(RuleType.COUNT_DISTINCT_REWRITE);
     }
 
-    private static class CountDistinctRewriter extends DefaultExpressionRewriter<Void> {
-        private static final CountDistinctRewriter INSTANCE = new CountDistinctRewriter();
-
-        public static Expression rewrite(Expression expr) {
-            return expr.accept(INSTANCE, null);
-        }
-
-        @Override
-        public Expression visitCount(Count count, Void context) {
-            if (count.isDistinct() && count.arity() == 1) {
-                Expression child = count.child(0);
-                if (child.getDataType().isBitmapType()) {
-                    return new BitmapUnionCount(child);
-                }
-                if (child.getDataType().isHllType()) {
-                    return new HllUnionAgg(child);
+    private static boolean containsCountObject(LogicalAggregate<Plan> agg) {
+        for (NamedExpression ne : agg.getOutputExpressions()) {
+            boolean needRewrite = ne.anyMatch(expr -> {
+                if (expr instanceof Count && ((Count) expr).isDistinct() && expr.arity() == 1) {
+                    DataType dataType = expr.child(0).getDataType();
+                    return dataType.isBitmapType() || dataType.isHllType();
                 }
+                return false;
+            });
+            if (needRewrite) {
+                return true;
             }
-            return count;
         }
+        return false;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CountLiteralRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CountLiteralRewrite.java
index dfe13b388f5..bfbd6599cf8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CountLiteralRewrite.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CountLiteralRewrite.java
@@ -27,13 +27,14 @@ import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableList.Builder;
 import com.google.common.collect.Lists;
 
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * count(1) ==> count(*)
@@ -50,21 +51,31 @@ public class CountLiteralRewrite extends OneRewriteRuleFactory {
                         return agg;
                     }
 
-                    Map<Boolean, List<NamedExpression>> projectsAndAggFunc = newExprs.stream()
-                            .collect(Collectors.partitioningBy(Expression::isConstant));
+                    List<NamedExpression> projectFuncs = Lists.newArrayListWithCapacity(newExprs.size());
+                    Builder<NamedExpression> aggFuncsBuilder
+                            = ImmutableList.builderWithExpectedSize(newExprs.size());
+                    for (NamedExpression newExpr : newExprs) {
+                        if (newExpr.isConstant()) {
+                            projectFuncs.add(newExpr);
+                        } else {
+                            aggFuncsBuilder.add(newExpr);
+                        }
+                    }
 
-                    if (projectsAndAggFunc.get(false).isEmpty()) {
+                    List<NamedExpression> aggFuncs = aggFuncsBuilder.build();
+                    if (aggFuncs.isEmpty()) {
                         // if there is no group by keys and other agg func, don't rewrite
                         return null;
                     } else {
                         // if there is group by keys, put count(null) in projects, such as
                         // project(0 as count(null))
                         // --Aggregate(k1, group by k1)
-                        Plan plan = agg.withAggOutput(projectsAndAggFunc.get(false));
-                        if (!projectsAndAggFunc.get(true).isEmpty()) {
-                            projectsAndAggFunc.get(false).stream().map(NamedExpression::toSlot)
-                                    .forEach(projectsAndAggFunc.get(true)::add);
-                            plan = new LogicalProject<>(projectsAndAggFunc.get(true), plan);
+                        Plan plan = agg.withAggOutput(aggFuncs);
+                        if (!projectFuncs.isEmpty()) {
+                            for (NamedExpression aggFunc : aggFuncs) {
+                                projectFuncs.add(aggFunc.toSlot());
+                            }
+                            plan = new LogicalProject<>(projectFuncs, plan);
                         }
                         return plan;
                     }
@@ -77,9 +88,11 @@ public class CountLiteralRewrite extends OneRewriteRuleFactory {
         for (Expression expr : oldExprs) {
             Map<Expression, Expression> replaced = new HashMap<>();
             Set<AggregateFunction> oldAggFuncSet = expr.collect(AggregateFunction.class::isInstance);
-            oldAggFuncSet.stream()
-                    .filter(this::isCountLiteral)
-                    .forEach(c -> replaced.put(c, rewrite((Count) c)));
+            for (AggregateFunction aggFun : oldAggFuncSet) {
+                if (isCountLiteral(aggFun)) {
+                    replaced.put(aggFun, rewrite((Count) aggFun));
+                }
+            }
             expr = expr.rewriteUp(s -> replaced.getOrDefault(s, s));
             changed |= !replaced.isEmpty();
             newExprs.add((NamedExpression) expr);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateFilter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateFilter.java
index a3de71a770e..ef9e418f58d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateFilter.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateFilter.java
@@ -43,10 +43,10 @@ public class EliminateFilter implements RewriteRuleFactory {
     @Override
     public List<Rule> buildRules() {
         return ImmutableList.of(logicalFilter().when(
-                filter -> filter.getConjuncts().stream().anyMatch(BooleanLiteral.class::isInstance))
+                filter -> ExpressionUtils.containsType(filter.getConjuncts(), BooleanLiteral.class))
                 .thenApply(ctx -> {
                     LogicalFilter<Plan> filter = ctx.root;
-                    ImmutableSet.Builder newConjuncts = ImmutableSet.builder();
+                    ImmutableSet.Builder<Expression> newConjuncts = ImmutableSet.builder();
                     for (Expression expression : filter.getConjuncts()) {
                         if (expression == BooleanLiteral.FALSE) {
                             return new LogicalEmptyRelation(ctx.statementContext.getNextRelationId(),
@@ -73,8 +73,7 @@ public class EliminateFilter implements RewriteRuleFactory {
                             new ExpressionRewriteContext(ctx.cascadesContext);
                     for (Expression expression : filter.getConjuncts()) {
                         Expression newExpr = ExpressionUtils.replace(expression, replaceMap);
-                        Expression foldExpression =
-                                FoldConstantRule.INSTANCE.rewrite(newExpr, context);
+                        Expression foldExpression = FoldConstantRule.evaluate(newExpr, context);
 
                         if (foldExpression == BooleanLiteral.FALSE) {
                             return new LogicalEmptyRelation(
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupBy.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupBy.java
index 3b95e9b44e0..109cff192f2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupBy.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupBy.java
@@ -20,6 +20,7 @@ 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.Alias;
+import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.IsNull;
 import org.apache.doris.nereids.trees.expressions.NamedExpression;
 import org.apache.doris.nereids.trees.expressions.Slot;
@@ -31,11 +32,14 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.If;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.util.ExpressionUtils;
 import org.apache.doris.nereids.util.PlanUtils;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.ImmutableSet.Builder;
+
 import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * Eliminate GroupBy.
@@ -45,39 +49,53 @@ public class EliminateGroupBy extends OneRewriteRuleFactory {
     @Override
     public Rule build() {
         return logicalAggregate()
-                .when(agg -> agg.getGroupByExpressions().stream().allMatch(expr -> expr instanceof Slot))
+                .when(agg -> ExpressionUtils.allMatch(agg.getGroupByExpressions(), Slot.class::isInstance))
                 .then(agg -> {
-                    Set<Slot> groupby = agg.getGroupByExpressions().stream().map(e -> (Slot) e)
-                            .collect(Collectors.toSet());
+                    List<Expression> groupByExpressions = agg.getGroupByExpressions();
+                    Builder<Slot> groupBySlots
+                            = ImmutableSet.builderWithExpectedSize(groupByExpressions.size());
+                    for (Expression groupByExpression : groupByExpressions) {
+                        groupBySlots.add((Slot) groupByExpression);
+                    }
                     Plan child = agg.child();
-                    boolean unique = child.getLogicalProperties().getFunctionalDependencies()
-                            .isUniqueAndNotNull(groupby);
+                    boolean unique = child.getLogicalProperties()
+                            .getFunctionalDependencies()
+                            .isUniqueAndNotNull(groupBySlots.build());
                     if (!unique) {
                         return null;
                     }
-                    Set<AggregateFunction> aggregateFunctions = agg.getAggregateFunctions();
-                    if (!aggregateFunctions.stream().allMatch(
-                            f -> (f instanceof Sum || f instanceof Count || f instanceof Min || f instanceof Max)
-                                    && (f.arity() == 1 && f.child(0) instanceof Slot))) {
-                        return null;
+                    for (AggregateFunction f : agg.getAggregateFunctions()) {
+                        if (!((f instanceof Sum || f instanceof Count || f instanceof Min || f instanceof Max)
+                                && (f.arity() == 1 && f.child(0) instanceof Slot))) {
+                            return null;
+                        }
                     }
+                    List<NamedExpression> outputExpressions = agg.getOutputExpressions();
+
+                    ImmutableList.Builder<NamedExpression> newOutput
+                            = ImmutableList.builderWithExpectedSize(outputExpressions.size());
 
-                    List<NamedExpression> newOutput = agg.getOutputExpressions().stream().map(ne -> {
+                    for (NamedExpression ne : outputExpressions) {
                         if (ne instanceof Alias && ne.child(0) instanceof AggregateFunction) {
                             AggregateFunction f = (AggregateFunction) ne.child(0);
                             if (f instanceof Sum || f instanceof Min || f instanceof Max) {
-                                return new Alias(ne.getExprId(), f.child(0), ne.getName());
+                                newOutput.add(new Alias(ne.getExprId(), f.child(0), ne.getName()));
                             } else if (f instanceof Count) {
-                                return (NamedExpression) ne.withChildren(
-                                        new If(new IsNull(f.child(0)), Literal.of(0), Literal.of(1)));
+                                newOutput.add((NamedExpression) ne.withChildren(
+                                        new If(
+                                            new IsNull(f.child(0)),
+                                            Literal.of(0),
+                                            Literal.of(1)
+                                        )
+                                ));
                             } else {
                                 throw new IllegalStateException("Unexpected aggregate function: " + f);
                             }
                         } else {
-                            return ne;
+                            newOutput.add(ne);
                         }
-                    }).collect(Collectors.toList());
-                    return PlanUtils.projectOrSelf(newOutput, child);
+                    }
+                    return PlanUtils.projectOrSelf(newOutput.build(), child);
                 }).toRule(RuleType.ELIMINATE_GROUP_BY);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateMarkJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateMarkJoin.java
index 2c5a4bbdd14..2e426beae46 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateMarkJoin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateMarkJoin.java
@@ -19,9 +19,11 @@ 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.rules.expression.ExpressionRewriteContext;
 import org.apache.doris.nereids.rules.expression.rules.TrySimplifyPredicateWithMarkJoinSlot;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
 import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
 import org.apache.doris.nereids.util.ExpressionUtils;
 
@@ -38,15 +40,22 @@ public class EliminateMarkJoin extends OneRewriteRuleFactory {
     public Rule build() {
         return logicalFilter(logicalJoin().when(
                 join -> join.getJoinType().isSemiJoin() && !join.getMarkJoinConjuncts().isEmpty()))
-                        .when(filter -> canSimplifyMarkJoin(filter.getConjuncts()))
-                        .then(filter -> filter.withChildren(eliminateMarkJoin(filter.child())))
+                        .when(filter -> canSimplifyMarkJoin(filter.getConjuncts(), null))
+                        .thenApply(ctx -> {
+                            LogicalFilter<LogicalJoin<Plan, Plan>> filter = ctx.root;
+                            ExpressionRewriteContext rewriteContext = new ExpressionRewriteContext(ctx.cascadesContext);
+                            if (canSimplifyMarkJoin(filter.getConjuncts(), rewriteContext)) {
+                                return filter.withChildren(eliminateMarkJoin(filter.child()));
+                            }
+                            return filter;
+                        })
                         .toRule(RuleType.ELIMINATE_MARK_JOIN);
     }
 
-    private boolean canSimplifyMarkJoin(Set<Expression> predicates) {
+    private boolean canSimplifyMarkJoin(Set<Expression> predicates, ExpressionRewriteContext rewriteContext) {
         return ExpressionUtils
                 .canInferNotNullForMarkSlot(TrySimplifyPredicateWithMarkJoinSlot.INSTANCE
-                        .rewrite(ExpressionUtils.and(predicates), null));
+                        .rewrite(ExpressionUtils.and(predicates), rewriteContext), rewriteContext);
     }
 
     private LogicalJoin<Plan, Plan> eliminateMarkJoin(LogicalJoin<Plan, Plan> join) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateNotNull.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateNotNull.java
index db95d1fefa0..22393cb55f8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateNotNull.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateNotNull.java
@@ -24,7 +24,6 @@ import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.IsNull;
 import org.apache.doris.nereids.trees.expressions.Not;
 import org.apache.doris.nereids.trees.expressions.Slot;
-import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
 import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
@@ -41,7 +40,6 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * Eliminate Predicate `is not null`, like
@@ -85,29 +83,34 @@ public class EliminateNotNull implements RewriteRuleFactory {
         // remove `name` (it's generated), remove `id` (because `id > 0` already contains it)
         Set<Expression> predicatesNotContainIsNotNull = Sets.newHashSet();
         List<Slot> slotsFromIsNotNull = Lists.newArrayList();
-        exprs.stream()
-                .filter(expr -> !(expr instanceof Not)
-                        || !((Not) expr).isGeneratedIsNotNull()) // remove generated `is not null`
-                .forEach(expr -> {
-                    Optional<Slot> notNullSlot = TypeUtils.isNotNull(expr);
-                    if (notNullSlot.isPresent()) {
-                        slotsFromIsNotNull.add(notNullSlot.get());
-                    } else {
-                        predicatesNotContainIsNotNull.add(expr);
-                    }
-                });
+
+        for (Expression expr : exprs) {
+            // remove generated `is not null`
+            if (!(expr instanceof Not) || !((Not) expr).isGeneratedIsNotNull()) {
+                Optional<Slot> notNullSlot = TypeUtils.isNotNull(expr);
+                if (notNullSlot.isPresent()) {
+                    slotsFromIsNotNull.add(notNullSlot.get());
+                } else {
+                    predicatesNotContainIsNotNull.add(expr);
+                }
+            }
+        }
+
         Set<Slot> inferNonNotSlots = ExpressionUtils.inferNotNullSlots(
                 predicatesNotContainIsNotNull, ctx);
 
-        Set<Expression> keepIsNotNull = slotsFromIsNotNull.stream()
-                .filter(ExpressionTrait::nullable)
-                .filter(slot -> !inferNonNotSlots.contains(slot))
-                .map(slot -> new Not(new IsNull(slot))).collect(Collectors.toSet());
+        ImmutableSet.Builder<Expression> keepIsNotNull
+                = ImmutableSet.builderWithExpectedSize(slotsFromIsNotNull.size());
+        for (Slot slot : slotsFromIsNotNull) {
+            if (slot.nullable() && !inferNonNotSlots.contains(slot)) {
+                keepIsNotNull.add(new Not(new IsNull(slot)));
+            }
+        }
 
         // merge predicatesNotContainIsNotNull and keepIsNotNull into a new List
         return ImmutableList.<Expression>builder()
                 .addAll(predicatesNotContainIsNotNull)
-                .addAll(keepIsNotNull)
+                .addAll(keepIsNotNull.build())
                 .build();
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateOrderByConstant.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateOrderByConstant.java
index 969d6e6b045..021cae2d653 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateOrderByConstant.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateOrderByConstant.java
@@ -33,13 +33,19 @@ public class EliminateOrderByConstant extends OneRewriteRuleFactory {
     @Override
     public Rule build() {
         return logicalSort().then(sort -> {
-            List<OrderKey> orderKeysWithoutConst = sort
-                    .getOrderKeys()
-                    .stream()
-                    .filter(k -> !(k.getExpr().isConstant()))
-                    .collect(ImmutableList.toImmutableList());
+            List<OrderKey> orderKeys = sort.getOrderKeys();
+            ImmutableList.Builder<OrderKey> orderKeysWithoutConstBuilder
+                    = ImmutableList.builderWithExpectedSize(orderKeys.size());
+            for (OrderKey orderKey : orderKeys) {
+                if (!orderKey.getExpr().isConstant()) {
+                    orderKeysWithoutConstBuilder.add(orderKey);
+                }
+            }
+            List<OrderKey> orderKeysWithoutConst = orderKeysWithoutConstBuilder.build();
             if (orderKeysWithoutConst.isEmpty()) {
                 return sort.child();
+            } else if (orderKeysWithoutConst.size() == orderKeys.size()) {
+                return sort;
             }
             return sort.withOrderKeys(orderKeysWithoutConst);
         }).toRule(RuleType.ELIMINATE_ORDER_BY_CONSTANT);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java
index 697eb8fa5a3..5ec0f0cd698 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java
@@ -22,6 +22,7 @@ import org.apache.doris.nereids.rules.RuleType;
 import org.apache.doris.nereids.trees.expressions.Alias;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.expressions.OrderExpression;
 import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.expressions.WindowExpression;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Avg;
@@ -30,6 +31,7 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.Min;
 import org.apache.doris.nereids.trees.expressions.functions.agg.NullableAggregateFunction;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
 import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 import org.apache.doris.nereids.trees.plans.logical.LogicalWindow;
 import org.apache.doris.nereids.util.ExpressionUtils;
 
@@ -50,75 +52,94 @@ public class ExtractAndNormalizeWindowExpression extends OneRewriteRuleFactory i
 
     @Override
     public Rule build() {
-        return logicalProject().when(project -> containsWindowExpression(project.getProjects())).then(project -> {
-            List<NamedExpression> outputs =
-                    ExpressionUtils.rewriteDownShortCircuit(project.getProjects(), output -> {
-                        if (output instanceof WindowExpression) {
-                            WindowExpression windowExpression = (WindowExpression) output;
-                            Expression expression = ((WindowExpression) output).getFunction();
-                            if (expression instanceof Sum || expression instanceof Max
-                                    || expression instanceof Min || expression instanceof Avg) {
-                                // sum, max, min and avg in window function should be always nullable
-                                windowExpression = ((WindowExpression) output)
-                                        .withFunction(((NullableAggregateFunction) expression)
-                                                .withAlwaysNullable(true));
+        return logicalProject()
+                .when(project -> ExpressionUtils.containsWindowExpression(project.getProjects()))
+                .then(this::normalize)
+                .toRule(RuleType.EXTRACT_AND_NORMALIZE_WINDOW_EXPRESSIONS);
+    }
+
+    private Plan normalize(LogicalProject<Plan> project) {
+        List<NamedExpression> outputs =
+                ExpressionUtils.rewriteDownShortCircuit(project.getProjects(), output -> {
+                    if (output instanceof WindowExpression) {
+                        WindowExpression windowExpression = (WindowExpression) output;
+                        Expression expression = ((WindowExpression) output).getFunction();
+                        if (expression instanceof Sum || expression instanceof Max
+                                || expression instanceof Min || expression instanceof Avg) {
+                            // sum, max, min and avg in window function should be always nullable
+                            windowExpression = ((WindowExpression) output)
+                                    .withFunction(
+                                            ((NullableAggregateFunction) expression).withAlwaysNullable(true)
+                                    );
+                        }
+
+                        ImmutableList.Builder<Expression> nonLiteralPartitionKeys =
+                                ImmutableList.builderWithExpectedSize(windowExpression.getPartitionKeys().size());
+                        for (Expression partitionKey : windowExpression.getPartitionKeys()) {
+                            if (!partitionKey.isConstant()) {
+                                nonLiteralPartitionKeys.add(partitionKey);
                             }
-                            // remove literal partition by and order by keys
-                            return windowExpression.withPartitionKeysOrderKeys(
-                                    windowExpression.getPartitionKeys().stream()
-                                            .filter(partitionExpr -> !partitionExpr.isConstant())
-                                            .collect(Collectors.toList()),
-                                    windowExpression.getOrderKeys().stream()
-                                            .filter(orderExpression -> !orderExpression
-                                                    .getOrderKey().getExpr().isConstant())
-                                            .collect(Collectors.toList()));
                         }
-                        return output;
-                    });
-
-            // 1. handle bottom projects
-            Set<Alias> existedAlias = ExpressionUtils.collect(outputs, Alias.class::isInstance);
-            Set<Expression> toBePushedDown = collectExpressionsToBePushedDown(outputs);
-            NormalizeToSlotContext context = NormalizeToSlotContext.buildContext(existedAlias, toBePushedDown);
-            // set toBePushedDown exprs as NamedExpression, e.g. (a+1) -> Alias(a+1)
-            Set<NamedExpression> bottomProjects = context.pushDownToNamedExpression(toBePushedDown);
-            Plan normalizedChild;
-            if (bottomProjects.isEmpty()) {
-                normalizedChild = project.child();
-            } else {
-                normalizedChild = project.withProjectsAndChild(
-                        ImmutableList.copyOf(bottomProjects), project.child());
-            }
-
-            // 2. handle window's outputs and windowExprs
-            // need to replace exprs with SlotReference in WindowSpec, due to LogicalWindow.getExpressions()
-
-            // because alias is pushed down to bottom project
-            // we need replace alias's child expr with corresponding alias's slot in output
-            // so create a customNormalizeMap alias's child -> alias.toSlot to do it
-            Map<Expression, Slot> customNormalizeMap = toBePushedDown.stream()
-                    .filter(expr -> expr instanceof Alias)
-                    .collect(Collectors.toMap(expr -> ((Alias) expr).child(), expr -> ((Alias) expr).toSlot(),
-                            (oldExpr, newExpr) -> oldExpr));
-
-            List<NamedExpression> normalizedOutputs = context.normalizeToUseSlotRef(outputs,
-                    (ctx, expr) -> customNormalizeMap.getOrDefault(expr, null));
-            Set<WindowExpression> normalizedWindows =
-                    ExpressionUtils.collect(normalizedOutputs, WindowExpression.class::isInstance);
-
-            existedAlias = ExpressionUtils.collect(normalizedOutputs, Alias.class::isInstance);
-            NormalizeToSlotContext ctxForWindows = NormalizeToSlotContext.buildContext(
-                    existedAlias, Sets.newHashSet(normalizedWindows));
-
-            Set<NamedExpression> normalizedWindowWithAlias = ctxForWindows.pushDownToNamedExpression(normalizedWindows);
-            // only need normalized windowExpressions
-            LogicalWindow normalizedLogicalWindow =
-                    new LogicalWindow<>(ImmutableList.copyOf(normalizedWindowWithAlias), normalizedChild);
-
-            // 3. handle top projects
-            List<NamedExpression> topProjects = ctxForWindows.normalizeToUseSlotRef(normalizedOutputs);
-            return project.withProjectsAndChild(topProjects, normalizedLogicalWindow);
-        }).toRule(RuleType.EXTRACT_AND_NORMALIZE_WINDOW_EXPRESSIONS);
+
+                        ImmutableList.Builder<OrderExpression> nonLiteralOrderExpressions =
+                                ImmutableList.builderWithExpectedSize(windowExpression.getOrderKeys().size());
+                        for (OrderExpression orderExpr : windowExpression.getOrderKeys()) {
+                            if (!orderExpr.getOrderKey().getExpr().isConstant()) {
+                                nonLiteralOrderExpressions.add(orderExpr);
+                            }
+                        }
+
+                        // remove literal partition by and order by keys
+                        return windowExpression.withPartitionKeysOrderKeys(
+                                nonLiteralPartitionKeys.build(),
+                                nonLiteralOrderExpressions.build()
+                        );
+                    }
+                    return output;
+                });
+
+        // 1. handle bottom projects
+        Set<Alias> existedAlias = ExpressionUtils.collect(outputs, Alias.class::isInstance);
+        Set<Expression> toBePushedDown = collectExpressionsToBePushedDown(outputs);
+        NormalizeToSlotContext context = NormalizeToSlotContext.buildContext(existedAlias, toBePushedDown);
+        // set toBePushedDown exprs as NamedExpression, e.g. (a+1) -> Alias(a+1)
+        Set<NamedExpression> bottomProjects = context.pushDownToNamedExpression(toBePushedDown);
+        Plan normalizedChild;
+        if (bottomProjects.isEmpty()) {
+            normalizedChild = project.child();
+        } else {
+            normalizedChild = project.withProjectsAndChild(
+                    ImmutableList.copyOf(bottomProjects), project.child());
+        }
+
+        // 2. handle window's outputs and windowExprs
+        // need to replace exprs with SlotReference in WindowSpec, due to LogicalWindow.getExpressions()
+
+        // because alias is pushed down to bottom project
+        // we need replace alias's child expr with corresponding alias's slot in output
+        // so create a customNormalizeMap alias's child -> alias.toSlot to do it
+        Map<Expression, Slot> customNormalizeMap = toBePushedDown.stream()
+                .filter(expr -> expr instanceof Alias)
+                .collect(Collectors.toMap(expr -> ((Alias) expr).child(), expr -> ((Alias) expr).toSlot(),
+                        (oldExpr, newExpr) -> oldExpr));
+
+        List<NamedExpression> normalizedOutputs = context.normalizeToUseSlotRef(outputs,
+                (ctx, expr) -> customNormalizeMap.getOrDefault(expr, null));
+        Set<WindowExpression> normalizedWindows =
+                ExpressionUtils.collect(normalizedOutputs, WindowExpression.class::isInstance);
+
+        existedAlias = ExpressionUtils.collect(normalizedOutputs, Alias.class::isInstance);
+        NormalizeToSlotContext ctxForWindows = NormalizeToSlotContext.buildContext(
+                existedAlias, Sets.newHashSet(normalizedWindows));
+
+        Set<NamedExpression> normalizedWindowWithAlias = ctxForWindows.pushDownToNamedExpression(normalizedWindows);
+        // only need normalized windowExpressions
+        LogicalWindow normalizedLogicalWindow =
+                new LogicalWindow<>(ImmutableList.copyOf(normalizedWindowWithAlias), normalizedChild);
+
+        // 3. handle top projects
+        List<NamedExpression> topProjects = ctxForWindows.normalizeToUseSlotRef(normalizedOutputs);
+        return project.withProjectsAndChild(topProjects, normalizedLogicalWindow);
     }
 
     private Set<Expression> collectExpressionsToBePushedDown(List<NamedExpression> expressions) {
@@ -161,10 +182,4 @@ public class ExtractAndNormalizeWindowExpression extends OneRewriteRuleFactory i
             })
             .collect(ImmutableSet.toImmutableSet());
     }
-
-    private boolean containsWindowExpression(List<NamedExpression> expressions) {
-        // WindowExpression in top LogicalProject will be normalized as Alias(SlotReference) after this rule,
-        // so it will not be normalized infinitely
-        return expressions.stream().anyMatch(expr -> expr.anyMatch(WindowExpression.class::isInstance));
-    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractSingleTableExpressionFromDisjunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractSingleTableExpressionFromDisjunction.java
index 4ecc79ae94e..2f8e1404b71 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractSingleTableExpressionFromDisjunction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractSingleTableExpressionFromDisjunction.java
@@ -30,7 +30,6 @@ import com.google.common.collect.Lists;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * Paper: Quantifying TPC-H Choke Points and Their Optimizations
@@ -84,13 +83,9 @@ public class ExtractSingleTableExpressionFromDisjunction extends OneRewriteRuleF
             }
             // only check table in first disjunct.
             // In our example, qualifiers = { n1, n2 }
-            Expression first = disjuncts.get(0);
-            Set<String> qualifiers = first.getInputSlots()
-                    .stream()
-                    .map(slot -> String.join(".", slot.getQualifier()))
-                    .collect(Collectors.toSet());
             // try to extract
-            for (String qualifier : qualifiers) {
+            for (Slot inputSlot : disjuncts.get(0).getInputSlots()) {
+                String qualifier = String.join(".", inputSlot.getQualifier());
                 List<Expression> extractForAll = Lists.newArrayList();
                 boolean success = true;
                 for (Expression expr : ExpressionUtils.extractDisjunction(conjunct)) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferJoinNotNull.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferJoinNotNull.java
index e7168ca0e99..39a0e63ff21 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferJoinNotNull.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferJoinNotNull.java
@@ -26,7 +26,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
 import org.apache.doris.nereids.util.ExpressionUtils;
 import org.apache.doris.nereids.util.PlanUtils;
 
-import java.util.HashSet;
+import java.util.LinkedHashSet;
 import java.util.Set;
 
 /**
@@ -46,7 +46,7 @@ public class InferJoinNotNull extends OneRewriteRuleFactory {
             .whenNot(LogicalJoin::isMarkJoin)
             .thenApply(ctx -> {
                 LogicalJoin<Plan, Plan> join = ctx.root;
-                Set<Expression> conjuncts = new HashSet<>();
+                Set<Expression> conjuncts = new LinkedHashSet<>();
                 conjuncts.addAll(join.getHashJoinConjuncts());
                 conjuncts.addAll(join.getOtherJoinConjuncts());
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeAggregate.java
index 3bdfbc582ac..9a0b9f8b5e0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeAggregate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeAggregate.java
@@ -201,7 +201,7 @@ public class MergeAggregate implements RewriteRuleFactory {
             return false;
         }
         // project cannot have expressions like a+1
-        if (ExpressionUtils.anyMatch(project.getProjects(),
+        if (ExpressionUtils.deapAnyMatch(project.getProjects(),
                 expr -> !(expr instanceof SlotReference) && !(expr instanceof Alias))) {
             return false;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeProjects.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeProjects.java
index d152178b523..3ea903f8565 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeProjects.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeProjects.java
@@ -20,9 +20,9 @@ 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.NamedExpression;
-import org.apache.doris.nereids.trees.expressions.WindowExpression;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+import org.apache.doris.nereids.util.ExpressionUtils;
 
 import java.util.List;
 
@@ -43,8 +43,8 @@ public class MergeProjects extends OneRewriteRuleFactory {
         // TODO modify ExtractAndNormalizeWindowExpression to handle nested window functions
         // here we just don't merge two projects if there is any window function
         return logicalProject(logicalProject())
-                .whenNot(project -> containsWindowExpression(project.getProjects())
-                        && containsWindowExpression(project.child().getProjects()))
+                .whenNot(project -> ExpressionUtils.containsWindowExpression(project.getProjects())
+                        && ExpressionUtils.containsWindowExpression(project.child().getProjects()))
                 .then(MergeProjects::mergeProjects).toRule(RuleType.MERGE_PROJECTS);
     }
 
@@ -54,8 +54,4 @@ public class MergeProjects extends OneRewriteRuleFactory {
         LogicalProject<?> newProject = childProject.canEliminate() ? project : childProject;
         return newProject.withProjectsAndChild(projectExpressions, childProject.child(0));
     }
-
-    private boolean containsWindowExpression(List<NamedExpression> expressions) {
-        return expressions.stream().anyMatch(expr -> expr.anyMatch(WindowExpression.class::isInstance));
-    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeSort.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeSort.java
index b36d0e63b85..b7554582885 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeSort.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeSort.java
@@ -24,13 +24,15 @@ import org.apache.doris.nereids.trees.expressions.Alias;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.NamedExpression;
 import org.apache.doris.nereids.trees.expressions.Slot;
+import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableList.Builder;
 import com.google.common.collect.Lists;
 
 import java.util.List;
-import java.util.stream.Stream;
 
 /**
  * SortNode on BE always output order keys because BE needs them to do merge sort. So we normalize LogicalSort as BE
@@ -40,29 +42,44 @@ import java.util.stream.Stream;
 public class NormalizeSort extends OneRewriteRuleFactory {
     @Override
     public Rule build() {
-        return logicalSort().whenNot(sort -> sort.getOrderKeys().stream()
-                        .map(OrderKey::getExpr).allMatch(Slot.class::isInstance))
+        return logicalSort().whenNot(this::allOrderKeyIsSlot)
                 .then(sort -> {
                     List<NamedExpression> newProjects = Lists.newArrayList();
-                    List<OrderKey> newOrderKeys = sort.getOrderKeys().stream()
-                            .map(orderKey -> {
-                                Expression expr = orderKey.getExpr();
-                                if (!(expr instanceof Slot)) {
-                                    Alias alias = new Alias(expr);
-                                    newProjects.add(alias);
-                                    expr = alias.toSlot();
-                                }
-                                return orderKey.withExpression(expr);
-                            }).collect(ImmutableList.toImmutableList());
-                    List<NamedExpression> bottomProjections = Stream.concat(
-                            sort.child().getOutput().stream(),
-                            newProjects.stream()
-                    ).collect(ImmutableList.toImmutableList());
-                    List<NamedExpression> topProjections = sort.getOutput().stream()
-                            .map(NamedExpression.class::cast)
-                            .collect(ImmutableList.toImmutableList());
-                    return new LogicalProject<>(topProjections, sort.withOrderKeysAndChild(newOrderKeys,
+
+                    Builder<OrderKey> newOrderKeys =
+                            ImmutableList.builderWithExpectedSize(sort.getOrderKeys().size());
+                    for (OrderKey orderKey : sort.getOrderKeys()) {
+                        Expression expr = orderKey.getExpr();
+                        if (!(expr instanceof Slot)) {
+                            Alias alias = new Alias(expr);
+                            newProjects.add(alias);
+                            expr = alias.toSlot();
+                            newOrderKeys.add(orderKey.withExpression(expr));
+                        } else {
+                            newOrderKeys.add(orderKey);
+                        }
+                    }
+
+                    List<Slot> childOutput = sort.child().getOutput();
+                    List<NamedExpression> bottomProjections = ImmutableList.<NamedExpression>builderWithExpectedSize(
+                            childOutput.size() + newProjects.size())
+                            .addAll(childOutput)
+                            .addAll(newProjects)
+                            .build();
+
+                    List<NamedExpression> topProjections = (List) sort.getOutput();
+                    return new LogicalProject<>(topProjections, sort.withOrderKeysAndChild(
+                            newOrderKeys.build(),
                             new LogicalProject<>(bottomProjections, sort.child())));
                 }).toRule(RuleType.NORMALIZE_SORT);
     }
+
+    private boolean allOrderKeyIsSlot(LogicalSort<Plan> sort) {
+        for (OrderKey orderKey : sort.getOrderKeys()) {
+            if (!(orderKey.getExpr() instanceof Slot)) {
+                return false;
+            }
+        }
+        return true;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeToSlot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeToSlot.java
index 683841a5f8f..ea2fb8f4beb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeToSlot.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeToSlot.java
@@ -104,15 +104,19 @@ public interface NormalizeToSlot {
          */
         public <E extends Expression> List<E> normalizeToUseSlotRef(Collection<E> expressions,
                 BiFunction<NormalizeToSlotContext, Expression, Expression> customNormalize) {
-            return expressions.stream()
-                    .map(expr -> (E) expr.rewriteDownShortCircuit(child -> {
-                        Expression newChild = customNormalize.apply(this, child);
-                        if (newChild != null && newChild != child) {
-                            return newChild;
-                        }
-                        NormalizeToSlotTriplet normalizeToSlotTriplet = normalizeToSlotMap.get(child);
-                        return normalizeToSlotTriplet == null ? child : normalizeToSlotTriplet.remainExpr;
-                    })).collect(ImmutableList.toImmutableList());
+            ImmutableList.Builder<E> result = ImmutableList.builderWithExpectedSize(expressions.size());
+            for (E expr : expressions) {
+                Expression rewriteExpr = expr.rewriteDownShortCircuit(child -> {
+                    Expression newChild = customNormalize.apply(this, child);
+                    if (newChild != null && newChild != child) {
+                        return newChild;
+                    }
+                    NormalizeToSlotTriplet normalizeToSlotTriplet = normalizeToSlotMap.get(child);
+                    return normalizeToSlotTriplet == null ? child : normalizeToSlotTriplet.remainExpr;
+                });
+                result.add((E) rewriteExpr);
+            }
+            return result.build();
         }
 
         public <E extends Expression> List<E> normalizeToUseSlotRefWithoutWindowFunction(
@@ -131,13 +135,20 @@ public interface NormalizeToSlot {
          * bottom: k1#0, (k2#1 + 1) AS (k2 + 1)#2;
          */
         public Set<NamedExpression> pushDownToNamedExpression(Collection<? extends Expression> needToPushExpressions) {
-            return needToPushExpressions.stream()
-                    .map(expr -> {
-                        NormalizeToSlotTriplet normalizeToSlotTriplet = normalizeToSlotMap.get(expr);
-                        return normalizeToSlotTriplet == null
-                                ? (NamedExpression) expr
-                                : normalizeToSlotTriplet.pushedExpr;
-                    }).collect(ImmutableSet.toImmutableSet());
+            ImmutableSet.Builder<NamedExpression> result
+                    = ImmutableSet.builderWithExpectedSize(needToPushExpressions.size());
+            for (Expression expr : needToPushExpressions) {
+                NormalizeToSlotTriplet normalizeToSlotTriplet = normalizeToSlotMap.get(expr);
+                result.add(normalizeToSlotTriplet == null
+                        ? (NamedExpression) expr
+                        : normalizeToSlotTriplet.pushedExpr);
+            }
+            return result.build();
+        }
+
+        public NamedExpression pushDownToNamedExpression(Expression expr) {
+            NormalizeToSlotTriplet normalizeToSlotTriplet = normalizeToSlotMap.get(expr);
+            return normalizeToSlotTriplet == null ? (NamedExpression) expr : normalizeToSlotTriplet.pushedExpr;
         }
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanPartition.java
index 60df874f2a1..0aacde1cc19 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanPartition.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanPartition.java
@@ -36,7 +36,6 @@ import org.apache.doris.nereids.util.Utils;
 import org.apache.doris.qe.ConnectContext;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -50,6 +49,23 @@ import java.util.stream.Collectors;
  * MergeConsecutiveProjects and all predicate push down related rules.
  */
 public class PruneOlapScanPartition implements RewriteRuleFactory {
+
+    @Override
+    public List<Rule> buildRules() {
+        return ImmutableList.of(
+                logicalFilter(logicalOlapScan())
+                        .when(p -> !p.child().isPartitionPruned())
+                        .thenApply(ctx -> prunePartitions(ctx.cascadesContext, ctx.root.child(), ctx.root))
+                        .toRule(RuleType.OLAP_SCAN_PARTITION_PRUNE),
+
+                logicalFilter(logicalProject(logicalOlapScan()))
+                        .when(p -> !p.child().child().isPartitionPruned())
+                        .when(p -> p.child().hasPushedDownToProjectionFunctions())
+                        .thenApply(ctx -> prunePartitions(ctx.cascadesContext, ctx.root.child().child(), ctx.root))
+                        .toRule(RuleType.OLAP_SCAN_WITH_PROJECT_PARTITION_PRUNE)
+        );
+    }
+
     private <T extends Plan> Plan prunePartitions(CascadesContext ctx,
                     LogicalOlapScan scan, LogicalFilter<T> originalFilter) {
         OlapTable table = scan.getTable();
@@ -59,20 +75,22 @@ public class PruneOlapScanPartition implements RewriteRuleFactory {
         }
 
         List<Slot> output = scan.getOutput();
-        Map<String, Slot> scanOutput = Maps.newHashMapWithExpectedSize(output.size() * 2);
-        for (Slot slot : output) {
-            scanOutput.put(slot.getName().toLowerCase(), slot);
-        }
-
         PartitionInfo partitionInfo = table.getPartitionInfo();
         List<Column> partitionColumns = partitionInfo.getPartitionColumns();
         List<Slot> partitionSlots = new ArrayList<>(partitionColumns.size());
         for (Column column : partitionColumns) {
-            Slot slot = scanOutput.get(column.getName().toLowerCase());
-            if (slot == null) {
+            Slot partitionSlot = null;
+            // loop search is faster than build a map
+            for (Slot slot : output) {
+                if (slot.getName().equalsIgnoreCase(column.getName())) {
+                    partitionSlot = slot;
+                    break;
+                }
+            }
+            if (partitionSlot == null) {
                 return originalFilter;
             } else {
-                partitionSlots.add(slot);
+                partitionSlots.add(partitionSlot);
             }
         }
 
@@ -105,19 +123,4 @@ public class PruneOlapScanPartition implements RewriteRuleFactory {
         }
         return originalFilter.withChildren(ImmutableList.of(rewrittenScan));
     }
-
-    @Override
-    public List<Rule> buildRules() {
-        return ImmutableList.of(
-                logicalFilter(logicalOlapScan()).when(p -> !p.child().isPartitionPruned()).thenApply(ctx -> {
-                    return prunePartitions(ctx.cascadesContext, ctx.root.child(), ctx.root);
-                }).toRule(RuleType.OLAP_SCAN_PARTITION_PRUNE),
-
-                logicalFilter(logicalProject(logicalOlapScan()))
-                        .when(p -> !p.child().child().isPartitionPruned())
-                        .when(p -> p.child().hasPushedDownToProjectionFunctions()).thenApply(ctx -> {
-                            return prunePartitions(ctx.cascadesContext, ctx.root.child().child(), ctx.root);
-                        }).toRule(RuleType.OLAP_SCAN_WITH_PROJECT_PARTITION_PRUNE)
-        );
-    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpPredicates.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpPredicates.java
index 26e1358c2e5..b02c51b1fe9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpPredicates.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpPredicates.java
@@ -31,16 +31,17 @@ import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
 import org.apache.doris.nereids.util.ExpressionUtils;
 
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.ImmutableSet.Builder;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-import java.util.Collection;
 import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.function.Supplier;
-import java.util.stream.Collectors;
 
 /**
  * poll up effective predicates from operator's children.
@@ -60,7 +61,7 @@ public class PullUpPredicates extends PlanVisitor<ImmutableSet<Expression>, Void
     @Override
     public ImmutableSet<Expression> visitLogicalFilter(LogicalFilter<? extends Plan> filter, Void context) {
         return cacheOrElse(filter, () -> {
-            List<Expression> predicates = Lists.newArrayList(filter.getConjuncts());
+            Set<Expression> predicates = Sets.newLinkedHashSet(filter.getConjuncts());
             predicates.addAll(filter.child().accept(this, context));
             return getAvailableExpressions(predicates, filter);
         });
@@ -82,14 +83,14 @@ public class PullUpPredicates extends PlanVisitor<ImmutableSet<Expression>, Void
     public ImmutableSet<Expression> visitLogicalProject(LogicalProject<? extends Plan> project, Void context) {
         return cacheOrElse(project, () -> {
             ImmutableSet<Expression> childPredicates = project.child().accept(this, context);
-
-            Set<Expression> allPredicates = Sets.newHashSet(childPredicates);
-            project.getAliasToProducer().forEach((k, v) -> {
-                Set<Expression> expressions = childPredicates.stream()
-                        .map(e -> e.rewriteDownShortCircuit(c -> c.equals(v) ? k : c)).collect(Collectors.toSet());
-                allPredicates.addAll(expressions);
-            });
-
+            Set<Expression> allPredicates = Sets.newLinkedHashSet(childPredicates);
+            for (Entry<Slot, Expression> kv : project.getAliasToProducer().entrySet()) {
+                Slot k = kv.getKey();
+                Expression v = kv.getValue();
+                for (Expression childPredicate : childPredicates) {
+                    allPredicates.add(childPredicate.rewriteDownShortCircuit(c -> c.equals(v) ? k : c));
+                }
+            }
             return getAvailableExpressions(allPredicates, project);
         });
     }
@@ -99,21 +100,22 @@ public class PullUpPredicates extends PlanVisitor<ImmutableSet<Expression>, Void
         return cacheOrElse(aggregate, () -> {
             ImmutableSet<Expression> childPredicates = aggregate.child().accept(this, context);
             // TODO
-            Map<Expression, Slot> expressionSlotMap = aggregate.getOutputExpressions()
-                    .stream()
-                    .filter(this::hasAgg)
-                    .collect(Collectors.toMap(
-                            namedExpr -> {
-                                if (namedExpr instanceof Alias) {
-                                    return ((Alias) namedExpr).child();
-                                } else {
-                                    return namedExpr;
-                                }
-                            }, NamedExpression::toSlot)
+            List<NamedExpression> outputExpressions = aggregate.getOutputExpressions();
+
+            Map<Expression, Slot> expressionSlotMap
+                    = Maps.newLinkedHashMapWithExpectedSize(outputExpressions.size());
+            for (NamedExpression output : outputExpressions) {
+                if (hasAgg(output)) {
+                    expressionSlotMap.putIfAbsent(
+                            output instanceof Alias ? output.child(0) : output, output.toSlot()
                     );
-            Expression expression = ExpressionUtils.replace(ExpressionUtils.and(Lists.newArrayList(childPredicates)),
-                    expressionSlotMap);
-            List<Expression> predicates = ExpressionUtils.extractConjunction(expression);
+                }
+            }
+            Expression expression = ExpressionUtils.replace(
+                    ExpressionUtils.and(Lists.newArrayList(childPredicates)),
+                    expressionSlotMap
+            );
+            Set<Expression> predicates = Sets.newLinkedHashSet(ExpressionUtils.extractConjunction(expression));
             return getAvailableExpressions(predicates, aggregate);
         });
     }
@@ -128,12 +130,23 @@ public class PullUpPredicates extends PlanVisitor<ImmutableSet<Expression>, Void
         return predicates;
     }
 
-    private ImmutableSet<Expression> getAvailableExpressions(Collection<Expression> predicates, Plan plan) {
-        Set<Expression> expressions = Sets.newHashSet(predicates);
-        expressions.addAll(PredicatePropagation.infer(expressions));
-        return expressions.stream()
-                .filter(p -> plan.getOutputSet().containsAll(p.getInputSlots()))
-                .collect(ImmutableSet.toImmutableSet());
+    private ImmutableSet<Expression> getAvailableExpressions(Set<Expression> predicates, Plan plan) {
+        Set<Expression> inferPredicates = PredicatePropagation.infer(predicates);
+        Builder<Expression> newPredicates = ImmutableSet.builderWithExpectedSize(predicates.size() + 10);
+        Set<Slot> outputSet = plan.getOutputSet();
+
+        for (Expression predicate : predicates) {
+            if (outputSet.containsAll(predicate.getInputSlots())) {
+                newPredicates.add(predicate);
+            }
+        }
+
+        for (Expression inferPredicate : inferPredicates) {
+            if (outputSet.containsAll(inferPredicate.getInputSlots())) {
+                newPredicates.add(inferPredicate);
+            }
+        }
+        return newPredicates.build();
     }
 
     private boolean hasAgg(Expression expression) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughAggregation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughAggregation.java
index f3a54fd8eea..798a41b3764 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughAggregation.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughAggregation.java
@@ -29,7 +29,7 @@ import org.apache.doris.nereids.util.PlanUtils;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Sets;
 
-import java.util.HashSet;
+import java.util.LinkedHashSet;
 import java.util.Set;
 
 /**
@@ -60,9 +60,9 @@ public class PushDownFilterThroughAggregation extends OneRewriteRuleFactory {
             LogicalAggregate<Plan> aggregate = filter.child();
             Set<Slot> canPushDownSlots = getCanPushDownSlots(aggregate);
 
-            Set<Expression> pushDownPredicates = Sets.newHashSet();
-            Set<Expression> filterPredicates = Sets.newHashSet();
-            filter.getConjuncts().forEach(conjunct -> {
+            Set<Expression> pushDownPredicates = Sets.newLinkedHashSet();
+            Set<Expression> filterPredicates = Sets.newLinkedHashSet();
+            for (Expression conjunct : filter.getConjuncts()) {
                 Set<Slot> conjunctSlots = conjunct.getInputSlots();
                 // NOTICE: filter not contain slot should not be pushed. e.g. 'a' = 'b'
                 if (!conjunctSlots.isEmpty() && canPushDownSlots.containsAll(conjunctSlots)) {
@@ -70,7 +70,7 @@ public class PushDownFilterThroughAggregation extends OneRewriteRuleFactory {
                 } else {
                     filterPredicates.add(conjunct);
                 }
-            });
+            }
             if (pushDownPredicates.isEmpty()) {
                 return null;
             }
@@ -84,7 +84,7 @@ public class PushDownFilterThroughAggregation extends OneRewriteRuleFactory {
      * get the slots that can be pushed down
      */
     public static Set<Slot> getCanPushDownSlots(LogicalAggregate<? extends Plan> aggregate) {
-        Set<Slot> canPushDownSlots = new HashSet<>();
+        Set<Slot> canPushDownSlots = new LinkedHashSet<>();
         if (aggregate.getSourceRepeat().isPresent()) {
             // When there is a repeat, the push-down condition is consistent with the repeat
             aggregate.getSourceRepeat().get().getCommonGroupingSetExpressions().stream()
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java
index 77c90820a25..71834a66b19 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java
@@ -22,7 +22,6 @@ import org.apache.doris.nereids.rules.Rule;
 import org.apache.doris.nereids.rules.RuleType;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.Slot;
-import org.apache.doris.nereids.trees.expressions.WindowExpression;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
 import org.apache.doris.nereids.trees.plans.logical.LogicalLimit;
@@ -49,16 +48,16 @@ public class PushDownFilterThroughProject implements RewriteRuleFactory {
     public List<Rule> buildRules() {
         return ImmutableList.of(
                 logicalFilter(logicalProject())
-                        .whenNot(filter -> filter.child().getProjects().stream().anyMatch(
-                                expr -> expr.anyMatch(WindowExpression.class::isInstance)))
+                        .whenNot(filter -> ExpressionUtils.containsWindowExpression(filter.child().getProjects()))
                         .whenNot(filter -> filter.child().hasPushedDownToProjectionFunctions())
                         .then(PushDownFilterThroughProject::pushdownFilterThroughProject)
                         .toRule(RuleType.PUSH_DOWN_FILTER_THROUGH_PROJECT),
                 // filter(project(limit)) will change to filter(limit(project)) by PushdownProjectThroughLimit,
                 // then we should change filter(limit(project)) to project(filter(limit))
                 logicalFilter(logicalLimit(logicalProject()))
-                        .whenNot(filter -> filter.child().child().getProjects().stream()
-                                .anyMatch(expr -> expr.anyMatch(WindowExpression.class::isInstance)))
+                        .whenNot(filter ->
+                                ExpressionUtils.containsWindowExpression(filter.child().child().getProjects())
+                        )
                         .whenNot(filter -> filter.child().child().hasPushedDownToProjectionFunctions())
                         .then(PushDownFilterThroughProject::pushdownFilterThroughLimitProject)
                         .toRule(RuleType.PUSH_DOWN_FILTER_THROUGH_PROJECT_UNDER_LIMIT)
@@ -111,14 +110,14 @@ public class PushDownFilterThroughProject implements RewriteRuleFactory {
             Set<Expression> conjuncts, Set<Slot> childOutputs) {
         Set<Expression> pushDownPredicates = Sets.newLinkedHashSet();
         Set<Expression> remainPredicates = Sets.newLinkedHashSet();
-        conjuncts.forEach(conjunct -> {
+        for (Expression conjunct : conjuncts) {
             Set<Slot> conjunctSlots = conjunct.getInputSlots();
             if (childOutputs.containsAll(conjunctSlots)) {
                 pushDownPredicates.add(conjunct);
             } else {
                 remainPredicates.add(conjunct);
             }
-        });
+        }
         return Pair.of(remainPredicates, pushDownPredicates);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ReorderJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ReorderJoin.java
index 97af548ce02..238db403be6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ReorderJoin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ReorderJoin.java
@@ -42,7 +42,7 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-import java.util.HashSet;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -293,10 +293,10 @@ public class ReorderJoin extends OneRewriteRuleFactory {
         }
 
         // following this multiJoin just contain INNER/CROSS.
-        Set<Expression> joinFilter = new HashSet<>(multiJoinHandleChildren.getJoinFilter());
+        Set<Expression> joinFilter = new LinkedHashSet<>(multiJoinHandleChildren.getJoinFilter());
 
         Plan left = multiJoinHandleChildren.child(0);
-        Set<Integer> usedPlansIndex = new HashSet<>();
+        Set<Integer> usedPlansIndex = new LinkedHashSet<>();
         usedPlansIndex.add(0);
 
         while (usedPlansIndex.size() != multiJoinHandleChildren.children().size()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SimplifyAggGroupBy.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SimplifyAggGroupBy.java
index cc0c7f12f33..6dc446d88ca 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SimplifyAggGroupBy.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/SimplifyAggGroupBy.java
@@ -19,16 +19,18 @@ 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.TreeNode;
 import org.apache.doris.nereids.trees.expressions.BinaryArithmetic;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.util.ExpressionUtils;
+import org.apache.doris.nereids.util.Utils;
 
-import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 
+import java.util.List;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 /**
  * Simplify Aggregate group by Multiple to One. For example
@@ -41,20 +43,25 @@ public class SimplifyAggGroupBy extends OneRewriteRuleFactory {
     @Override
     public Rule build() {
         return logicalAggregate()
-                .when(agg -> agg.getGroupByExpressions().size() > 1)
-                .when(agg -> agg.getGroupByExpressions().stream().allMatch(this::isBinaryArithmeticSlot))
+                .when(agg -> agg.getGroupByExpressions().size() > 1
+                        && ExpressionUtils.allMatch(agg.getGroupByExpressions(), this::isBinaryArithmeticSlot))
                 .then(agg -> {
-                    Set<Expression> slots = agg.getGroupByExpressions().stream()
-                            .flatMap(e -> e.getInputSlots().stream()).collect(Collectors.toSet());
+                    List<Expression> groupByExpressions = agg.getGroupByExpressions();
+                    ImmutableSet.Builder<Expression> inputSlots
+                            = ImmutableSet.builderWithExpectedSize(groupByExpressions.size());
+                    for (Expression groupByExpression : groupByExpressions) {
+                        inputSlots.addAll(groupByExpression.getInputSlots());
+                    }
+                    Set<Expression> slots = inputSlots.build();
                     if (slots.size() != 1) {
                         return null;
                     }
-                    return agg.withGroupByAndOutput(ImmutableList.copyOf(slots), agg.getOutputExpressions());
+                    return agg.withGroupByAndOutput(Utils.fastToImmutableList(slots), agg.getOutputExpressions());
                 })
                 .toRule(RuleType.SIMPLIFY_AGG_GROUP_BY);
     }
 
-    private boolean isBinaryArithmeticSlot(Expression expr) {
+    private boolean isBinaryArithmeticSlot(TreeNode<Expression> expr) {
         if (expr instanceof Slot) {
             return true;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java
index 7ca697726ee..e69bffb301b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java
@@ -434,7 +434,12 @@ public abstract class AbstractSelectMaterializedIndexRule {
     }
 
     protected static boolean preAggEnabledByHint(LogicalOlapScan olapScan) {
-        return olapScan.getHints().stream().anyMatch("PREAGGOPEN"::equalsIgnoreCase);
+        for (String hint : olapScan.getHints()) {
+            if ("PREAGGOPEN".equalsIgnoreCase(hint)) {
+                return true;
+            }
+        }
+        return false;
     }
 
     public static String normalizeName(String name) {
@@ -447,11 +452,11 @@ public abstract class AbstractSelectMaterializedIndexRule {
     }
 
     protected SlotContext generateBaseScanExprToMvExpr(LogicalOlapScan mvPlan) {
-        Map<Slot, Slot> baseSlotToMvSlot = new HashMap<>();
-        Map<String, Slot> mvNameToMvSlot = new HashMap<>();
         if (mvPlan.getSelectedIndexId() == mvPlan.getTable().getBaseIndexId()) {
-            return new SlotContext(baseSlotToMvSlot, mvNameToMvSlot, new TreeSet<Expression>());
+            return SlotContext.EMPTY;
         }
+        Map<Slot, Slot> baseSlotToMvSlot = new HashMap<>();
+        Map<String, Slot> mvNameToMvSlot = new HashMap<>();
         for (Slot mvSlot : mvPlan.getOutputByIndex(mvPlan.getSelectedIndexId())) {
             boolean isPushed = false;
             for (Slot baseSlot : mvPlan.getOutput()) {
@@ -505,6 +510,8 @@ public abstract class AbstractSelectMaterializedIndexRule {
 
     /** SlotContext */
     protected static class SlotContext {
+        public static final SlotContext EMPTY
+                = new SlotContext(ImmutableMap.of(), ImmutableMap.of(), ImmutableSet.of());
 
         // base index Slot to selected mv Slot
         public final Map<Slot, Slot> baseSlotToMvSlot;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java
index bd3494378ae..b1a06e3875a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java
@@ -74,8 +74,10 @@ import org.apache.doris.nereids.types.BigIntType;
 import org.apache.doris.nereids.types.DataType;
 import org.apache.doris.nereids.types.VarcharType;
 import org.apache.doris.nereids.util.ExpressionUtils;
+import org.apache.doris.nereids.util.Utils;
 import org.apache.doris.planner.PlanNode;
 
+import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
@@ -256,9 +258,10 @@ public class SelectMaterializedIndexWithAggregate extends AbstractSelectMaterial
                             if (result.indexId == scan.getTable().getBaseIndexId()) {
                                 LogicalOlapScan mvPlanWithoutAgg = SelectMaterializedIndexWithoutAggregate.select(scan,
                                         project::getInputSlots, filter::getConjuncts,
-                                        Stream.concat(filter.getExpressions().stream(),
-                                                project.getExpressions().stream())
-                                                .collect(ImmutableSet.toImmutableSet()));
+                                        Suppliers.memoize(() -> Utils.concatToSet(
+                                                filter.getExpressions(), project.getExpressions()
+                                        ))
+                                );
                                 SlotContext slotContextWithoutAgg = generateBaseScanExprToMvExpr(mvPlanWithoutAgg);
 
                                 return agg.withChildren(new LogicalProject(
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithoutAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithoutAggregate.java
index 7960dd73df9..e05a1eda3e6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithoutAggregate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithoutAggregate.java
@@ -32,7 +32,9 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
 import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
 import org.apache.doris.nereids.util.ExpressionUtils;
+import org.apache.doris.nereids.util.Utils;
 
+import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
@@ -42,7 +44,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
-import java.util.stream.Stream;
 
 /**
  * Select materialized index, i.e., both for rollup and materialized view when aggregate is not present.
@@ -70,11 +71,13 @@ public class SelectMaterializedIndexWithoutAggregate extends AbstractSelectMater
 
                             LogicalOlapScan mvPlan = select(
                                     scan, project::getInputSlots, filter::getConjuncts,
-                                    Stream.concat(filter.getExpressions().stream(),
-                                        project.getExpressions().stream()).collect(ImmutableSet.toImmutableSet()));
+                                    Suppliers.memoize(() ->
+                                            Utils.concatToSet(filter.getExpressions(), project.getExpressions())
+                                    )
+                            );
                             SlotContext slotContext = generateBaseScanExprToMvExpr(mvPlan);
 
-                            return new LogicalProject(
+                            return new LogicalProject<>(
                                     generateProjectsAlias(project.getOutput(), slotContext),
                                     new ReplaceExpressions(slotContext).replace(
                                         project.withChildren(filter.withChildren(mvPlan)), mvPlan));
@@ -90,7 +93,7 @@ public class SelectMaterializedIndexWithoutAggregate extends AbstractSelectMater
 
                             LogicalOlapScan mvPlan = select(
                                     scan, project::getInputSlots, ImmutableSet::of,
-                                    new HashSet<>(project.getExpressions()));
+                                    () -> Utils.fastToImmutableSet(project.getExpressions()));
                             SlotContext slotContext = generateBaseScanExprToMvExpr(mvPlan);
 
                             return new LogicalProject(
@@ -107,8 +110,10 @@ public class SelectMaterializedIndexWithoutAggregate extends AbstractSelectMater
                             LogicalOlapScan scan = filter.child();
                             LogicalOlapScan mvPlan = select(
                                     scan, filter::getOutputSet, filter::getConjuncts,
-                                    Stream.concat(filter.getExpressions().stream(),
-                                            filter.getOutputSet().stream()).collect(ImmutableSet.toImmutableSet()));
+                                    Suppliers.memoize(() ->
+                                            Utils.concatToSet(filter.getExpressions(), filter.getOutputSet())
+                                    )
+                            );
                             SlotContext slotContext = generateBaseScanExprToMvExpr(mvPlan);
 
                             return new LogicalProject(
@@ -127,7 +132,8 @@ public class SelectMaterializedIndexWithoutAggregate extends AbstractSelectMater
 
                             LogicalOlapScan mvPlan = select(
                                     scan, project::getInputSlots, ImmutableSet::of,
-                                    new HashSet<>(project.getExpressions()));
+                                    Suppliers.memoize(() -> Utils.fastToImmutableSet(project.getExpressions()))
+                            );
                             SlotContext slotContext = generateBaseScanExprToMvExpr(mvPlan);
 
                             return new LogicalProject(
@@ -145,7 +151,7 @@ public class SelectMaterializedIndexWithoutAggregate extends AbstractSelectMater
 
                             LogicalOlapScan mvPlan = select(
                                     scan, scan::getOutputSet, ImmutableSet::of,
-                                    scan.getOutputSet());
+                                    () -> (Set) scan.getOutputSet());
                             SlotContext slotContext = generateBaseScanExprToMvExpr(mvPlan);
 
                             return new LogicalProject(
@@ -169,7 +175,7 @@ public class SelectMaterializedIndexWithoutAggregate extends AbstractSelectMater
             LogicalOlapScan scan,
             Supplier<Set<Slot>> requiredScanOutputSupplier,
             Supplier<Set<Expression>> predicatesSupplier,
-            Set<? extends Expression> requiredExpr) {
+            Supplier<Set<Expression>> requiredExpr) {
         OlapTable table = scan.getTable();
         long baseIndexId = table.getBaseIndexId();
         KeysType keysType = scan.getTable().getKeysType();
@@ -186,21 +192,24 @@ public class SelectMaterializedIndexWithoutAggregate extends AbstractSelectMater
                 throw new RuntimeException("Not supported keys type: " + keysType);
         }
 
-        Set<Slot> requiredSlots = new HashSet<>();
-        requiredSlots.addAll(requiredScanOutputSupplier.get());
-        requiredSlots.addAll(ExpressionUtils.getInputSlotSet(requiredExpr));
-        requiredSlots.addAll(ExpressionUtils.getInputSlotSet(predicatesSupplier.get()));
+        Supplier<Set<Slot>> requiredSlots = Suppliers.memoize(() -> {
+            Set<Slot> set = new HashSet<>();
+            set.addAll(requiredScanOutputSupplier.get());
+            set.addAll(ExpressionUtils.getInputSlotSet(requiredExpr.get()));
+            set.addAll(ExpressionUtils.getInputSlotSet(predicatesSupplier.get()));
+            return set;
+        });
         if (scan.getTable().isDupKeysOrMergeOnWrite()) {
             // Set pre-aggregation to `on` to keep consistency with legacy logic.
             List<MaterializedIndex> candidates = scan
                     .getTable().getVisibleIndex().stream().filter(index -> index.getId() != baseIndexId)
                     .filter(index -> !indexHasAggregate(index, scan)).filter(index -> containAllRequiredColumns(index,
-                            scan, requiredScanOutputSupplier.get(), requiredExpr, predicatesSupplier.get()))
+                            scan, requiredScanOutputSupplier.get(), requiredExpr.get(), predicatesSupplier.get()))
                     .collect(Collectors.toList());
             long bestIndex = selectBestIndex(candidates, scan, predicatesSupplier.get());
             // this is fail-safe for select mv
             // select baseIndex if bestIndex's slots' data types are different from baseIndex
-            bestIndex = isSameDataType(scan, bestIndex, requiredSlots) ? bestIndex : baseIndexId;
+            bestIndex = isSameDataType(scan, bestIndex, requiredSlots.get()) ? bestIndex : baseIndexId;
             return scan.withMaterializedIndexSelected(PreAggStatus.on(), bestIndex);
         } else {
             final PreAggStatus preAggStatus;
@@ -221,7 +230,7 @@ public class SelectMaterializedIndexWithoutAggregate extends AbstractSelectMater
             List<MaterializedIndex> candidates = table.getVisibleIndex().stream()
                     .filter(index -> table.getKeyColumnsByIndexId(index.getId()).size() == baseIndexKeySize)
                     .filter(index -> containAllRequiredColumns(index, scan, requiredScanOutputSupplier.get(),
-                            requiredExpr, predicatesSupplier.get()))
+                            requiredExpr.get(), predicatesSupplier.get()))
                     .collect(Collectors.toList());
 
             if (candidates.size() == 1) {
@@ -231,7 +240,7 @@ public class SelectMaterializedIndexWithoutAggregate extends AbstractSelectMater
                 long bestIndex = selectBestIndex(candidates, scan, predicatesSupplier.get());
                 // this is fail-safe for select mv
                 // select baseIndex if bestIndex's slots' data types are different from baseIndex
-                bestIndex = isSameDataType(scan, bestIndex, requiredSlots) ? bestIndex : baseIndexId;
+                bestIndex = isSameDataType(scan, bestIndex, requiredSlots.get()) ? bestIndex : baseIndexId;
                 return scan.withMaterializedIndexSelected(preAggStatus, bestIndex);
             }
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
index 2ed1afc5677..57a79037d80 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
@@ -132,6 +132,7 @@ import org.apache.doris.statistics.Statistics;
 import org.apache.doris.statistics.StatisticsBuilder;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.commons.collections.CollectionUtils;
@@ -753,8 +754,14 @@ public class StatsCalculator extends DefaultPlanVisitor<Statistics, Void> {
     //       2. Consider the influence of runtime filter
     //       3. Get NDV and column data size from StatisticManger, StatisticManager doesn't support it now.
     private Statistics computeCatalogRelation(CatalogRelation catalogRelation) {
-        Set<SlotReference> slotSet = catalogRelation.getOutput().stream().filter(SlotReference.class::isInstance)
-                .map(s -> (SlotReference) s).collect(Collectors.toSet());
+        List<Slot> output = catalogRelation.getOutput();
+        ImmutableSet.Builder<SlotReference> slotSetBuilder = ImmutableSet.builderWithExpectedSize(output.size());
+        for (Slot slot : output) {
+            if (slot instanceof SlotReference) {
+                slotSetBuilder.add((SlotReference) slot);
+            }
+        }
+        Set<SlotReference> slotSet = slotSetBuilder.build();
         Map<Expression, ColumnStatistic> columnStatisticMap = new HashMap<>();
         TableIf table = catalogRelation.getTable();
         double rowCount = catalogRelation.getTable().getRowCountForNereids();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java
index 59d2acbe22b..92bbcdb9b38 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java
@@ -17,9 +17,12 @@
 
 package org.apache.doris.nereids.trees;
 
+import org.apache.doris.nereids.util.MutableState;
+import org.apache.doris.nereids.util.MutableState.EmptyMutableState;
 import org.apache.doris.nereids.util.Utils;
 
 import java.util.List;
+import java.util.Optional;
 
 /**
  * Abstract class for plan node in Nereids, include plan node and expression.
@@ -30,8 +33,13 @@ import java.util.List;
 public abstract class AbstractTreeNode<NODE_TYPE extends TreeNode<NODE_TYPE>>
         implements TreeNode<NODE_TYPE> {
     protected final List<NODE_TYPE> children;
-    // TODO: Maybe we should use a GroupPlan to avoid TreeNode hold the GroupExpression.
-    // https://github.com/apache/doris/pull/9807#discussion_r884829067
+
+    // this field is special, because other fields in tree node is immutable, but in some scenes, mutable
+    // state is necessary. e.g. the rewrite framework need distinguish whether the plan is created by
+    // rules, the framework can set this field to a state variable to quickly judge without new big plan.
+    // we should avoid using it as much as possible, because mutable state is easy to cause bugs and
+    // difficult to locate.
+    private MutableState mutableState = EmptyMutableState.INSTANCE;
 
     protected AbstractTreeNode(NODE_TYPE... children) {
         // NOTE: ImmutableList.copyOf has additional clone of the list, so here we
@@ -55,6 +63,16 @@ public abstract class AbstractTreeNode<NODE_TYPE extends TreeNode<NODE_TYPE>>
         return children;
     }
 
+    @Override
+    public <T> Optional<T> getMutableState(String key) {
+        return mutableState.get(key);
+    }
+
+    @Override
+    public void setMutableState(String key, Object state) {
+        this.mutableState = this.mutableState.set(key, state);
+    }
+
     public int arity() {
         return children.size();
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java
index d37070865e2..6d1a298eb79 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java
@@ -28,11 +28,13 @@ import java.util.ArrayList;
 import java.util.Deque;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Optional;
 import java.util.Set;
 import java.util.function.BiFunction;
 import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.function.Predicate;
+import java.util.function.Supplier;
 
 /**
  * interface for all node in Nereids, include plan node and expression.
@@ -48,6 +50,21 @@ public interface TreeNode<NODE_TYPE extends TreeNode<NODE_TYPE>> {
 
     int arity();
 
+    <T> Optional<T> getMutableState(String key);
+
+    /** getOrInitMutableState */
+    default <T> T getOrInitMutableState(String key, Supplier<T> initState) {
+        Optional<T> mutableState = getMutableState(key);
+        if (!mutableState.isPresent()) {
+            T state = initState.get();
+            setMutableState(key, state);
+            return state;
+        }
+        return mutableState.get();
+    }
+
+    void setMutableState(String key, Object value);
+
     default NODE_TYPE withChildren(NODE_TYPE... children) {
         return withChildren(Utils.fastToImmutableList(children));
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/BinaryOperator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/BinaryOperator.java
index 01a61d576d2..750f3a77881 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/BinaryOperator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/BinaryOperator.java
@@ -24,7 +24,6 @@ import org.apache.doris.nereids.types.DataType;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
-import java.util.Objects;
 
 /**
  * Abstract for all binary operator, include binary arithmetic, compound predicate, comparison predicate.
@@ -63,9 +62,4 @@ public abstract class BinaryOperator extends Expression implements BinaryExpress
     public String shapeInfo() {
         return "(" + left().shapeInfo() + " " + symbol + " " + right().shapeInfo() + ")";
     }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(symbol, left(), right());
-    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ComparisonPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ComparisonPredicate.java
index c9d10bde36d..d343f6f9356 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ComparisonPredicate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ComparisonPredicate.java
@@ -61,10 +61,10 @@ public abstract class ComparisonPredicate extends BinaryOperator {
 
     @Override
     public void checkLegalityBeforeTypeCoercion() {
-        children().forEach(c -> {
+        for (Expression c : children) {
             if (c.getDataType().isComplexType() && !c.getDataType().isArrayType()) {
                 throw new AnalysisException("comparison predicate could not contains complex type: " + this.toSql());
             }
-        });
+        }
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java
index a7947c82a56..75cef0fc946 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Expression.java
@@ -39,6 +39,7 @@ import org.apache.doris.nereids.types.DataType;
 import org.apache.doris.nereids.types.MapType;
 import org.apache.doris.nereids.types.StructField;
 import org.apache.doris.nereids.types.StructType;
+import org.apache.doris.nereids.util.ExpressionUtils;
 import org.apache.doris.nereids.util.Utils;
 
 import com.google.common.base.Preconditions;
@@ -70,20 +71,43 @@ public abstract class Expression extends AbstractTreeNode<Expression> implements
 
     protected Expression(Expression... children) {
         super(children);
-        int maxChildDepth = 0;
-        int sumChildWidth = 0;
+
         boolean hasUnbound = false;
-        boolean compareWidthAndDepth = true;
-        for (int i = 0; i < children.length; ++i) {
-            Expression child = children[i];
-            maxChildDepth = Math.max(child.depth, maxChildDepth);
-            sumChildWidth += child.width;
-            hasUnbound |= child.hasUnbound;
-            compareWidthAndDepth &= (child.compareWidthAndDepth & child.supportCompareWidthAndDepth());
+        switch (children.length) {
+            case 0:
+                this.depth = 1;
+                this.width = 1;
+                this.compareWidthAndDepth = supportCompareWidthAndDepth();
+                break;
+            case 1:
+                Expression child = children[0];
+                this.depth = child.depth + 1;
+                this.width = child.width;
+                this.compareWidthAndDepth = child.compareWidthAndDepth && supportCompareWidthAndDepth();
+                break;
+            case 2:
+                Expression left = children[0];
+                Expression right = children[1];
+                this.depth = Math.max(left.depth, right.depth) + 1;
+                this.width = left.width + right.width;
+                this.compareWidthAndDepth =
+                        left.compareWidthAndDepth && right.compareWidthAndDepth && supportCompareWidthAndDepth();
+                break;
+            default:
+                int maxChildDepth = 0;
+                int sumChildWidth = 0;
+                boolean compareWidthAndDepth = true;
+                for (Expression expression : children) {
+                    child = expression;
+                    maxChildDepth = Math.max(child.depth, maxChildDepth);
+                    sumChildWidth += child.width;
+                    hasUnbound |= child.hasUnbound;
+                    compareWidthAndDepth &= child.compareWidthAndDepth;
+                }
+                this.depth = maxChildDepth + 1;
+                this.width = sumChildWidth;
+                this.compareWidthAndDepth = compareWidthAndDepth;
         }
-        this.depth = maxChildDepth + 1;
-        this.width = sumChildWidth + ((children.length == 0) ? 1 : 0);
-        this.compareWidthAndDepth = compareWidthAndDepth;
 
         checkLimit();
         this.inferred = false;
@@ -96,20 +120,43 @@ public abstract class Expression extends AbstractTreeNode<Expression> implements
 
     protected Expression(List<Expression> children, boolean inferred) {
         super(children);
-        int maxChildDepth = 0;
-        int sumChildWidth = 0;
+
         boolean hasUnbound = false;
-        boolean compareWidthAndDepth = true;
-        for (int i = 0; i < children.size(); ++i) {
-            Expression child = children.get(i);
-            maxChildDepth = Math.max(child.depth, maxChildDepth);
-            sumChildWidth += child.width;
-            hasUnbound |= child.hasUnbound;
-            compareWidthAndDepth &= (child.compareWidthAndDepth & child.supportCompareWidthAndDepth());
+        switch (children.size()) {
+            case 0:
+                this.depth = 1;
+                this.width = 1;
+                this.compareWidthAndDepth = supportCompareWidthAndDepth();
+                break;
+            case 1:
+                Expression child = children.get(0);
+                this.depth = child.depth + 1;
+                this.width = child.width;
+                this.compareWidthAndDepth = child.compareWidthAndDepth && supportCompareWidthAndDepth();
+                break;
+            case 2:
+                Expression left = children.get(0);
+                Expression right = children.get(1);
+                this.depth = Math.max(left.depth, right.depth) + 1;
+                this.width = left.width + right.width;
+                this.compareWidthAndDepth =
+                        left.compareWidthAndDepth && right.compareWidthAndDepth && supportCompareWidthAndDepth();
+                break;
+            default:
+                int maxChildDepth = 0;
+                int sumChildWidth = 0;
+                boolean compareWidthAndDepth = true;
+                for (Expression expression : children) {
+                    child = expression;
+                    maxChildDepth = Math.max(child.depth, maxChildDepth);
+                    sumChildWidth += child.width;
+                    hasUnbound |= child.hasUnbound;
+                    compareWidthAndDepth &= child.compareWidthAndDepth;
+                }
+                this.depth = maxChildDepth + 1;
+                this.width = sumChildWidth;
+                this.compareWidthAndDepth = compareWidthAndDepth && supportCompareWidthAndDepth();
         }
-        this.depth = maxChildDepth + 1;
-        this.width = sumChildWidth + ((children.isEmpty()) ? 1 : 0);
-        this.compareWidthAndDepth = compareWidthAndDepth;
 
         checkLimit();
         this.inferred = inferred;
@@ -284,7 +331,7 @@ public abstract class Expression extends AbstractTreeNode<Expression> implements
         if (this instanceof LeafExpression) {
             return this instanceof Literal;
         } else {
-            return !(this instanceof Nondeterministic) && children().stream().allMatch(Expression::isConstant);
+            return !(this instanceof Nondeterministic) && ExpressionUtils.allMatch(children(), Expression::isConstant);
         }
     }
 
@@ -376,7 +423,7 @@ public abstract class Expression extends AbstractTreeNode<Expression> implements
 
     @Override
     public int hashCode() {
-        return 0;
+        return getClass().hashCode();
     }
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/InPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/InPredicate.java
index bcebdca4f5b..53a753c4535 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/InPredicate.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/InPredicate.java
@@ -28,6 +28,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList.Builder;
 
+import java.util.Collection;
 import java.util.List;
 import java.util.Objects;
 import java.util.stream.Collectors;
@@ -42,13 +43,13 @@ public class InPredicate extends Expression {
     private final Expression compareExpr;
     private final List<Expression> options;
 
-    public InPredicate(Expression compareExpr, List<Expression> options) {
+    public InPredicate(Expression compareExpr, Collection<Expression> options) {
         super(new Builder<Expression>().add(compareExpr).addAll(options).build());
         this.compareExpr = Objects.requireNonNull(compareExpr, "Compare Expr cannot be null");
         this.options = ImmutableList.copyOf(Objects.requireNonNull(options, "In list cannot be null"));
     }
 
-    public InPredicate(Expression compareExpr, List<Expression> options, boolean inferred) {
+    public InPredicate(Expression compareExpr, Collection<Expression> options, boolean inferred) {
         super(new Builder<Expression>().add(compareExpr).addAll(options).build(), inferred);
         this.compareExpr = Objects.requireNonNull(compareExpr, "Compare Expr cannot be null");
         this.options = ImmutableList.copyOf(Objects.requireNonNull(options, "In list cannot be null"));
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java
index 7cfaad72a2c..76083645512 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java
@@ -132,8 +132,8 @@ public class SlotReference extends Slot {
      */
     public static SlotReference fromColumn(TableIf table, Column column, List<String> qualifier, Relation relation) {
         DataType dataType = DataType.fromCatalogType(column.getType());
-        SlotReference slot = new SlotReference(StatementScopeIdGenerator.newExprId(), column.getName(), dataType,
-                column.isAllowNull(), qualifier, table, column, Optional.empty(), null);
+        SlotReference slot = new SlotReference(StatementScopeIdGenerator.newExprId(), () -> column.getName(), dataType,
+                column.isAllowNull(), qualifier, table, column, () -> Optional.of(column.getName()), null);
         if (relation != null && ConnectContext.get() != null
                 && ConnectContext.get().getStatementContext() != null) {
             ConnectContext.get().getStatementContext().addSlotToRelation(slot, relation);
@@ -260,6 +260,9 @@ public class SlotReference extends Slot {
 
     @Override
     public SlotReference withName(String name) {
+        if (this.name.get().equals(name)) {
+            return this;
+        }
         return new SlotReference(
                 exprId, () -> name, dataType, nullable, qualifier, table, column, internalName, subColPath);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignatureHelper.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignatureHelper.java
index 6d0a5d85de5..2cdbe43c12e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignatureHelper.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/ComputeSignatureHelper.java
@@ -38,6 +38,8 @@ import org.apache.doris.nereids.util.ResponsibilityChain;
 import org.apache.doris.nereids.util.TypeCoercionUtils;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableList.Builder;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
@@ -413,9 +415,12 @@ public class ComputeSignatureHelper {
             return signature;
         }
         DateTimeV2Type argType = finalType;
-        List<DataType> newArgTypes = signature.argumentsTypes.stream()
-                .map(at -> TypeCoercionUtils.replaceDateTimeV2WithTarget(at, argType))
-                .collect(Collectors.toList());
+
+        ImmutableList.Builder<DataType> newArgTypesBuilder = ImmutableList.builderWithExpectedSize(signature.arity);
+        for (DataType at : signature.argumentsTypes) {
+            newArgTypesBuilder.add(TypeCoercionUtils.replaceDateTimeV2WithTarget(at, argType));
+        }
+        List<DataType> newArgTypes = newArgTypesBuilder.build();
         signature = signature.withArgumentTypes(signature.hasVarArgs, newArgTypes);
         signature = signature.withArgumentTypes(signature.hasVarArgs, newArgTypes);
         if (signature.returnType instanceof DateTimeV2Type) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java
index 4f53b383d24..e45d3fb4da8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java
@@ -108,11 +108,18 @@ public abstract class AggregateFunction extends BoundFunction implements Expects
 
     @Override
     public String toSql() throws UnboundException {
-        String args = children()
-                .stream()
-                .map(Expression::toSql)
-                .collect(Collectors.joining(", "));
-        return getName() + "(" + (distinct ? "DISTINCT " : "") + args + ")";
+        StringBuilder sql = new StringBuilder(getName()).append("(");
+        if (distinct) {
+            sql.append("DISTINCT ");
+        }
+        int arity = arity();
+        for (int i = 0; i < arity; i++) {
+            sql.append(child(i).toSql());
+            if (i + 1 < arity) {
+                sql.append(", ");
+            }
+        }
+        return sql.append(")").toString();
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/PushDownToProjectionFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/PushDownToProjectionFunction.java
index 81678153cd6..d8e3642c36f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/PushDownToProjectionFunction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/PushDownToProjectionFunction.java
@@ -43,10 +43,9 @@ public abstract class PushDownToProjectionFunction extends ScalarFunction {
      */
     public static boolean validToPushDown(Expression pushDownExpr) {
         // Currently only element at for variant type could be pushed down
-        return pushDownExpr != null && !pushDownExpr.collectToList(
-                    PushDownToProjectionFunction.class::isInstance).stream().filter(
-                            x -> ((Expression) x).getDataType().isVariantType()).collect(
-                    Collectors.toList()).isEmpty();
+        return pushDownExpr != null && pushDownExpr.anyMatch(expr ->
+            expr instanceof PushDownToProjectionFunction && ((Expression) expr).getDataType().isVariantType()
+        );
     }
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java
index a33cc32c16f..38951ea9e45 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java
@@ -31,6 +31,7 @@ import org.apache.doris.nereids.util.StandardDateFormat;
 
 import com.google.common.collect.ImmutableSet;
 
+import java.time.LocalDate;
 import java.time.LocalDateTime;
 import java.time.Year;
 import java.time.temporal.ChronoField;
@@ -158,7 +159,9 @@ public class DateLiteral extends Literal {
 
     static String normalize(String s) {
         // merge consecutive space
-        s = s.replaceAll(" +", " ");
+        if (s.contains("  ")) {
+            s = s.replaceAll(" +", " ");
+        }
 
         StringBuilder sb = new StringBuilder();
 
@@ -261,6 +264,14 @@ public class DateLiteral extends Literal {
     }
 
     protected static TemporalAccessor parse(String s) {
+        // fast parse '2022-01-01'
+        if (s.length() == 10 && s.charAt(4) == '-' && s.charAt(7) == '-') {
+            TemporalAccessor date = fastParseDate(s);
+            if (date != null) {
+                return date;
+            }
+        }
+
         String originalString = s;
         try {
             TemporalAccessor dateTime;
@@ -477,4 +488,30 @@ public class DateLiteral extends Literal {
             return toEndOfTheDay();
         }
     }
+
+    private static TemporalAccessor fastParseDate(String date) {
+        Integer year = readNextInt(date, 0, 4);
+        Integer month = readNextInt(date, 5, 2);
+        Integer day = readNextInt(date, 8, 2);
+        if (year != null && month != null && day != null) {
+            return LocalDate.of(year, month, day);
+        } else {
+            return null;
+        }
+    }
+
+    private static Integer readNextInt(String str, int offset, int readLength) {
+        int value = 0;
+        int realReadLength = 0;
+        for (int i = offset; i < str.length(); i++) {
+            char c = str.charAt(i);
+            if ('0' <= c && c <= '9') {
+                realReadLength++;
+                value = value * 10 + (c - '0');
+            } else {
+                break;
+            }
+        }
+        return readLength == realReadLength ? value : null;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/DefaultExpressionRewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/DefaultExpressionRewriter.java
index 2248666dbca..fd25f9368ef 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/DefaultExpressionRewriter.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/DefaultExpressionRewriter.java
@@ -34,13 +34,13 @@ public abstract class DefaultExpressionRewriter<C> extends ExpressionVisitor<Exp
     }
 
     /** rewriteChildren */
-    public static final <C> Expression rewriteChildren(
-            ExpressionVisitor<Expression, C> rewriter, Expression expr, C context) {
+    public static final <E extends Expression, C> E rewriteChildren(
+            ExpressionVisitor<Expression, C> rewriter, E expr, C context) {
         switch (expr.arity()) {
             case 1: {
                 Expression originChild = expr.child(0);
                 Expression newChild = originChild.accept(rewriter, context);
-                return (originChild != newChild) ? expr.withChildren(ImmutableList.of(newChild)) : expr;
+                return (originChild != newChild) ? (E) expr.withChildren(ImmutableList.of(newChild)) : expr;
             }
             case 2: {
                 Expression originLeft = expr.child(0);
@@ -48,7 +48,7 @@ public abstract class DefaultExpressionRewriter<C> extends ExpressionVisitor<Exp
                 Expression originRight = expr.child(1);
                 Expression newRight = originRight.accept(rewriter, context);
                 return (originLeft != newLeft || originRight != newRight)
-                        ? expr.withChildren(ImmutableList.of(newLeft, newRight))
+                        ? (E) expr.withChildren(ImmutableList.of(newLeft, newRight))
                         : expr;
             }
             case 0: {
@@ -64,7 +64,7 @@ public abstract class DefaultExpressionRewriter<C> extends ExpressionVisitor<Exp
                     }
                     newChildren.add(newChild);
                 }
-                return hasNewChildren ? expr.withChildren(newChildren.build()) : expr;
+                return hasNewChildren ? (E) expr.withChildren(newChildren.build()) : expr;
             }
         }
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java
index 4be6d35dc94..286a92aab76 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java
@@ -28,7 +28,6 @@ import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.nereids.util.MutableState;
-import org.apache.doris.nereids.util.MutableState.EmptyMutableState;
 import org.apache.doris.nereids.util.TreeStringUtils;
 import org.apache.doris.statistics.Statistics;
 
@@ -58,13 +57,6 @@ public abstract class AbstractPlan extends AbstractTreeNode<Plan> implements Pla
     protected final Optional<GroupExpression> groupExpression;
     protected final Supplier<LogicalProperties> logicalPropertiesSupplier;
 
-    // this field is special, because other fields in tree node is immutable, but in some scenes, mutable
-    // state is necessary. e.g. the rewrite framework need distinguish whether the plan is created by
-    // rules, the framework can set this field to a state variable to quickly judge without new big plan.
-    // we should avoid using it as much as possible, because mutable state is easy to cause bugs and
-    // difficult to locate.
-    private MutableState mutableState = EmptyMutableState.INSTANCE;
-
     /**
      * all parameter constructor.
      */
@@ -108,7 +100,15 @@ public abstract class AbstractPlan extends AbstractTreeNode<Plan> implements Pla
 
     @Override
     public boolean canBind() {
-        return !bound() && children().stream().allMatch(Plan::bound);
+        if (bound()) {
+            return false;
+        }
+        for (Plan child : children()) {
+            if (!child.bound()) {
+                return false;
+            }
+        }
+        return true;
     }
 
     /**
@@ -185,16 +185,6 @@ public abstract class AbstractPlan extends AbstractTreeNode<Plan> implements Pla
         }
     }
 
-    @Override
-    public Optional<Object> getMutableState(String key) {
-        return mutableState.get(key);
-    }
... 3344 lines suppressed ...


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


(doris) 02/04: [fix](Nereids) fix link children failed (#33134)

Posted by yi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yiguolei pushed a commit to branch 2.1-tmp
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 38d580dfb79db993118b8c70a999aab983bd1931
Author: 924060929 <92...@qq.com>
AuthorDate: Tue Apr 2 10:24:44 2024 +0800

    [fix](Nereids) fix link children failed (#33134)
    
    #32617 introduce a bug: rewrite may not working when plan's arity >= 3.
    this pr fix it
    
    (cherry picked from commit 8b070d1a9d43aa7d25225a79da81573c384ee825)
---
 .../nereids/jobs/rewrite/PlanTreeRewriteJob.java   |  7 ++--
 .../nereids/trees/plans/SetOperationTest.java      | 38 ++++++++++++++++++++++
 2 files changed, 42 insertions(+), 3 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java
index 5e5acc29f66..c2b136c40fa 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java
@@ -110,16 +110,17 @@ public abstract class PlanTreeRewriteJob extends Job {
                 }
             }
             default: {
-                boolean changed = false;
+                boolean anyChanged = false;
                 int i = 0;
                 Plan[] newChildren = new Plan[childrenContext.length];
                 for (Plan oldChild : children) {
                     Plan result = childrenContext[i].result;
-                    changed = result != null && result != oldChild;
+                    boolean changed = result != null && result != oldChild;
                     newChildren[i] = changed ? result : oldChild;
+                    anyChanged |= changed;
                     i++;
                 }
-                return changed ? plan.withChildren(newChildren) : plan;
+                return anyChanged ? plan.withChildren(newChildren) : plan;
             }
         }
     }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/SetOperationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/SetOperationTest.java
index fa7fcddc3f6..b6932f84669 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/SetOperationTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/SetOperationTest.java
@@ -17,9 +17,19 @@
 
 package org.apache.doris.nereids.trees.plans;
 
+import org.apache.doris.nereids.analyzer.UnboundAlias;
+import org.apache.doris.nereids.analyzer.UnboundFunction;
+import org.apache.doris.nereids.analyzer.UnboundOneRowRelation;
+import org.apache.doris.nereids.trees.expressions.Alias;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Concat;
+import org.apache.doris.nereids.trees.expressions.literal.StringLiteral;
+import org.apache.doris.nereids.trees.plans.algebra.SetOperation.Qualifier;
+import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation;
+import org.apache.doris.nereids.trees.plans.logical.LogicalUnion;
 import org.apache.doris.nereids.util.PlanChecker;
 import org.apache.doris.utframe.TestWithFeService;
 
+import com.google.common.collect.ImmutableList;
 import org.junit.jupiter.api.Test;
 
 public class SetOperationTest extends TestWithFeService {
@@ -110,4 +120,32 @@ public class SetOperationTest extends TestWithFeService {
         PlanChecker.from(connectContext)
                 .checkPlannerResult("select 1, 2 union all select 1, 2 union all select 10 e, 20 f;");
     }
+
+    @Test
+    public void testUnion6() {
+        LogicalOneRowRelation first = new LogicalOneRowRelation(
+                RelationId.createGenerator().getNextId(), ImmutableList.of(
+                        new Alias(new Concat(new StringLiteral("1"), new StringLiteral("1")))
+        ));
+
+        UnboundOneRowRelation second = new UnboundOneRowRelation(
+                RelationId.createGenerator().getNextId(), ImmutableList.of(
+                    new UnboundAlias(new UnboundFunction(
+                            "concat",
+                            ImmutableList.of(new StringLiteral("2"), new StringLiteral("2")))
+                    )
+        ));
+
+        LogicalOneRowRelation third = new LogicalOneRowRelation(
+                RelationId.createGenerator().getNextId(), ImmutableList.of(
+                    new Alias(new Concat(new StringLiteral("3"), new StringLiteral("3")))
+        ));
+
+        LogicalUnion union = new LogicalUnion(Qualifier.ALL, ImmutableList.of(
+                first, second, third
+        ));
+        PlanChecker.from(connectContext, union)
+                .analyze()
+                .rewrite();
+    }
 }


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