-
Notifications
You must be signed in to change notification settings - Fork 5.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
plan: merge continuous selections and delete surely true expressions #24214
Changes from all commits
4538034
270aef2
81831b8
04f8dfc
f1c02a4
7c1fca3
c17abc5
9d09627
1aaf935
fbc5655
2808a10
7081de3
bba4de6
d073d1c
11bc14c
c3003a5
4b35833
d5ffd67
f029763
bcc6eaa
c227ca1
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,6 +24,7 @@ import ( | |
"github.com/pingcap/tidb/config" | ||
"github.com/pingcap/tidb/expression" | ||
"github.com/pingcap/tidb/infoschema" | ||
"github.com/pingcap/tidb/kv" | ||
"github.com/pingcap/tidb/lock" | ||
"github.com/pingcap/tidb/planner/property" | ||
"github.com/pingcap/tidb/privilege" | ||
|
@@ -156,9 +157,34 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic | |
return finalPlan, cost, nil | ||
} | ||
|
||
// mergeContinuousSelections merge continuous selections which may occur after changing plans. | ||
func mergeContinuousSelections(p PhysicalPlan) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why we need this in the physical phase? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Taking the issue as an example, a projection is eleminated in the postOptimize, then two selections become continuous, so we should merge at the last of postOptimize. |
||
if sel, ok := p.(*PhysicalSelection); ok { | ||
for { | ||
childSel := sel.children[0] | ||
if tmp, ok := childSel.(*PhysicalSelection); ok { | ||
sel.Conditions = append(sel.Conditions, tmp.Conditions...) | ||
sel.SetChild(0, tmp.children[0]) | ||
} else { | ||
break | ||
} | ||
} | ||
} | ||
for _, child := range p.Children() { | ||
mergeContinuousSelections(child) | ||
} | ||
// merge continuous selections in a coprocessor task of tiflash | ||
tableReader, isTableReader := p.(*PhysicalTableReader) | ||
if isTableReader && tableReader.StoreType == kv.TiFlash { | ||
mergeContinuousSelections(tableReader.tablePlan) | ||
tableReader.TablePlans = flattenPushDownPlan(tableReader.tablePlan) | ||
} | ||
} | ||
|
||
func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan { | ||
plan = eliminatePhysicalProjection(plan) | ||
plan = InjectExtraProjection(plan) | ||
mergeContinuousSelections(plan) | ||
plan = eliminateUnionScanAndLock(sctx, plan) | ||
plan = enableParallelApply(sctx, plan) | ||
return plan | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -42,6 +42,12 @@ func addSelection(p LogicalPlan, child LogicalPlan, conditions []expression.Expr | |
p.Children()[chIdx] = dual | ||
return | ||
} | ||
|
||
conditions = DeleteTrueExprs(p, conditions) | ||
if len(conditions) == 0 { | ||
p.Children()[chIdx] = child | ||
return | ||
} | ||
selection := LogicalSelection{Conditions: conditions}.Init(p.SCtx(), p.SelectBlockOffset()) | ||
selection.SetChildren(child) | ||
p.Children()[chIdx] = selection | ||
|
@@ -73,6 +79,8 @@ func splitSetGetVarFunc(filters []expression.Expression) ([]expression.Expressio | |
|
||
// PredicatePushDown implements LogicalPlan PredicatePushDown interface. | ||
func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { | ||
predicates = DeleteTrueExprs(p, predicates) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's a little redundant to remove it every time by checking the predicates passed down. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. en, we delete that in |
||
p.Conditions = DeleteTrueExprs(p, p.Conditions) | ||
canBePushDown, canNotBePushDown := splitSetGetVarFunc(p.Conditions) | ||
retConditions, child := p.children[0].PredicatePushDown(append(canBePushDown, predicates...)) | ||
retConditions = append(retConditions, canNotBePushDown...) | ||
|
@@ -100,6 +108,7 @@ func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression) | |
// PredicatePushDown implements LogicalPlan PredicatePushDown interface. | ||
func (ds *DataSource) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { | ||
predicates = expression.PropagateConstant(ds.ctx, predicates) | ||
predicates = DeleteTrueExprs(ds, predicates) | ||
ds.allConds = predicates | ||
ds.pushedDownConds, predicates = expression.PushDownExprs(ds.ctx.GetSessionVars().StmtCtx, predicates, ds.ctx.GetClient(), kv.UnSpecified) | ||
return predicates, ds | ||
|
@@ -532,6 +541,28 @@ func Conds2TableDual(p LogicalPlan, conds []expression.Expression) LogicalPlan { | |
return nil | ||
} | ||
|
||
// DeleteTrueExprs deletes the surely true expressions | ||
func DeleteTrueExprs(p LogicalPlan, conds []expression.Expression) []expression.Expression { | ||
newConds := make([]expression.Expression, 0, len(conds)) | ||
for _, cond := range conds { | ||
con, ok := cond.(*expression.Constant) | ||
if !ok { | ||
newConds = append(newConds, cond) | ||
continue | ||
} | ||
if expression.ContainMutableConst(p.SCtx(), []expression.Expression{con}) { | ||
newConds = append(newConds, cond) | ||
continue | ||
} | ||
sc := p.SCtx().GetSessionVars().StmtCtx | ||
if isTrue, err := con.Value.ToBool(sc); err == nil && isTrue == 1 { | ||
continue | ||
} | ||
newConds = append(newConds, cond) | ||
} | ||
return newConds | ||
} | ||
|
||
// outerJoinPropConst propagates constant equal and column equal conditions over outer join. | ||
func (p *LogicalJoin) outerJoinPropConst(predicates []expression.Expression) []expression.Expression { | ||
outerTable := p.children[0] | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
as the selection is eliminated after optimized.