Skip to content

Commit

Permalink
planner: move rule_constant_propagation to rule pkg. (pingcap#55231)
Browse files Browse the repository at this point in the history
  • Loading branch information
qingfeng777 committed Aug 11, 2024
1 parent 6df7aba commit 52a90a5
Show file tree
Hide file tree
Showing 7 changed files with 49 additions and 29 deletions.
1 change: 0 additions & 1 deletion pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,6 @@ go_library(
"rule_aggregation_skew_rewrite.go",
"rule_collect_plan_stats.go",
"rule_column_pruning.go",
"rule_constant_propagation.go",
"rule_decorrelate.go",
"rule_derive_topn_from_window.go",
"rule_eliminate_projection.go",
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/logical_selection.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,7 @@ func (p *LogicalSelection) PullUpConstantPredicates() []expression.Expression {
var result []expression.Expression
for _, candidatePredicate := range p.Conditions {
// the candidate predicate should be a constant and compare predicate
match := validCompareConstantPredicate(p.SCtx().GetExprCtx().GetEvalCtx(), candidatePredicate)
match := ruleutil.ValidCompareConstantPredicate(p.SCtx().GetExprCtx().GetEvalCtx(), candidatePredicate)
if match {
result = append(result, candidatePredicate)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ var optRuleList = []base.LogicalOptRule{
&SkewDistinctAggRewriter{},
&ProjectionEliminator{},
&MaxMinEliminator{},
&ConstantPropagationSolver{},
&rule.ConstantPropagationSolver{},
&ConvertOuterToInnerJoin{},
&PPDSolver{},
&OuterJoinEliminator{},
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/rule/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ go_library(
name = "rule",
srcs = [
"rule_build_key_info.go",
"rule_constant_propagation.go",
"rule_init.go",
],
importpath = "github.com/pingcap/tidb/pkg/planner/core/rule",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,13 +12,11 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package core
package rule

import (
"context"

"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
)
Expand Down Expand Up @@ -87,26 +85,3 @@ func (cp *ConstantPropagationSolver) execOptimize(currentPlan base.LogicalPlan,
func (*ConstantPropagationSolver) Name() string {
return "constant_propagation"
}

// validComparePredicate checks if the predicate is an expression like [column '>'|'>='|'<'|'<='|'=' constant].
// return param1: return true, if the predicate is a compare constant predicate.
// return param2: return the column side of predicate.
func validCompareConstantPredicate(ctx expression.EvalContext, candidatePredicate expression.Expression) bool {
scalarFunction, ok := candidatePredicate.(*expression.ScalarFunction)
if !ok {
return false
}
if scalarFunction.FuncName.L != ast.GT && scalarFunction.FuncName.L != ast.GE &&
scalarFunction.FuncName.L != ast.LT && scalarFunction.FuncName.L != ast.LE &&
scalarFunction.FuncName.L != ast.EQ {
return false
}
column, _ := expression.ValidCompareConstantPredicateHelper(ctx, scalarFunction, true)
if column == nil {
column, _ = expression.ValidCompareConstantPredicateHelper(ctx, scalarFunction, false)
}
if column == nil {
return false
}
return true
}
2 changes: 2 additions & 0 deletions pkg/planner/core/rule/util/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,11 +5,13 @@ go_library(
srcs = [
"build_key_info_misc.go",
"misc.go",
"rule_constant_propagation.go",
],
importpath = "github.com/pingcap/tidb/pkg/planner/core/rule/util",
visibility = ["//visibility:public"],
deps = [
"//pkg/expression",
"//pkg/parser/ast",
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/planner/core/base",
Expand Down
43 changes: 43 additions & 0 deletions pkg/planner/core/rule/util/rule_constant_propagation.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
// Copyright 2023 PingCAP, Inc.
//
// Licensed 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 util

import (
"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/parser/ast"
)

// ValidCompareConstantPredicate checks if the predicate is an expression like [column '>'|'>='|'<'|'<='|'=' constant].
// return param1: return true, if the predicate is a compare constant predicate.
// return param2: return the column side of predicate.
func ValidCompareConstantPredicate(ctx expression.EvalContext, candidatePredicate expression.Expression) bool {
scalarFunction, ok := candidatePredicate.(*expression.ScalarFunction)
if !ok {
return false
}
if scalarFunction.FuncName.L != ast.GT && scalarFunction.FuncName.L != ast.GE &&
scalarFunction.FuncName.L != ast.LT && scalarFunction.FuncName.L != ast.LE &&
scalarFunction.FuncName.L != ast.EQ {
return false
}
column, _ := expression.ValidCompareConstantPredicateHelper(ctx, scalarFunction, true)
if column == nil {
column, _ = expression.ValidCompareConstantPredicateHelper(ctx, scalarFunction, false)
}
if column == nil {
return false
}
return true
}

0 comments on commit 52a90a5

Please sign in to comment.