Skip to content

Commit 7df4f66

Browse files
authored
planner: classify logical union all into a separate file for later pkg move (#54299)
ref #51664, ref #52714
1 parent f0a5b1f commit 7df4f66

9 files changed

+197
-120
lines changed

pkg/planner/core/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ go_library(
2929
"logical_plans.go",
3030
"logical_sort.go",
3131
"logical_top_n.go",
32+
"logical_union_all.go",
3233
"memtable_predicate_extractor.go",
3334
"mock.go",
3435
"optimizer.go",

pkg/planner/core/exhaust_physical_plans.go

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3589,8 +3589,7 @@ func getLockPhysicalPlans(p *LogicalLock, prop *property.PhysicalProperty) ([]ba
35893589
return []base.PhysicalPlan{lock}, true, nil
35903590
}
35913591

3592-
// ExhaustPhysicalPlans implements LogicalPlan interface.
3593-
func (p *LogicalUnionAll) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
3592+
func exhaustUnionAllPhysicalPlans(p *LogicalUnionAll, prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
35943593
// TODO: UnionAll can not pass any order, but we can change it to sort merge to keep order.
35953594
if !prop.IsSortItemEmpty() || (prop.IsFlashProp() && prop.TaskTp != property.MppTaskType) {
35963595
return nil, true, nil

pkg/planner/core/logical_initialize.go

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -80,12 +80,6 @@ func (p LogicalExpand) Init(ctx base.PlanContext, offset int) *LogicalExpand {
8080
return &p
8181
}
8282

83-
// Init initializes LogicalUnionAll.
84-
func (p LogicalUnionAll) Init(ctx base.PlanContext, offset int) *LogicalUnionAll {
85-
p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeUnion, &p, offset)
86-
return &p
87-
}
88-
8983
// Init initializes LogicalPartitionUnionAll.
9084
func (p LogicalPartitionUnionAll) Init(ctx base.PlanContext, offset int) *LogicalPartitionUnionAll {
9185
p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypePartitionUnion, &p, offset)

pkg/planner/core/logical_plans.go

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1644,11 +1644,6 @@ func (p *LogicalIndexScan) getPKIsHandleCol(schema *expression.Schema) *expressi
16441644
return getPKIsHandleColFromSchema(p.Columns, schema, p.Source.TableInfo.PKIsHandle)
16451645
}
16461646

1647-
// LogicalUnionAll represents LogicalUnionAll plan.
1648-
type LogicalUnionAll struct {
1649-
logicalop.LogicalSchemaProducer
1650-
}
1651-
16521647
// LogicalPartitionUnionAll represents the LogicalUnionAll plan is for partition table.
16531648
type LogicalPartitionUnionAll struct {
16541649
LogicalUnionAll

pkg/planner/core/logical_union_all.go

Lines changed: 195 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,195 @@
1+
// Copyright 2024 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package core
16+
17+
import (
18+
"github.com/pingcap/tidb/pkg/expression"
19+
"github.com/pingcap/tidb/pkg/planner/core/base"
20+
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
21+
"github.com/pingcap/tidb/pkg/planner/property"
22+
"github.com/pingcap/tidb/pkg/planner/util"
23+
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
24+
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace/logicaltrace"
25+
"github.com/pingcap/tidb/pkg/planner/util/utilfuncp"
26+
"github.com/pingcap/tidb/pkg/util/plancodec"
27+
)
28+
29+
// LogicalUnionAll represents LogicalUnionAll plan.
30+
type LogicalUnionAll struct {
31+
logicalop.LogicalSchemaProducer
32+
}
33+
34+
// Init initializes LogicalUnionAll.
35+
func (p LogicalUnionAll) Init(ctx base.PlanContext, offset int) *LogicalUnionAll {
36+
p.BaseLogicalPlan = logicalop.NewBaseLogicalPlan(ctx, plancodec.TypeUnion, &p, offset)
37+
return &p
38+
}
39+
40+
// *************************** start implementation of logicalPlan interface ***************************
41+
42+
// HashCode inherits BaseLogicalPlan.LogicalPlan.<0th> implementation.
43+
44+
// PredicatePushDown implements base.LogicalPlan.<1st> interface.
45+
func (p *LogicalUnionAll) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan) {
46+
for i, proj := range p.Children() {
47+
newExprs := make([]expression.Expression, 0, len(predicates))
48+
newExprs = append(newExprs, predicates...)
49+
retCond, newChild := proj.PredicatePushDown(newExprs, opt)
50+
utilfuncp.AddSelection(p, newChild, retCond, i, opt)
51+
}
52+
return nil, p
53+
}
54+
55+
// PruneColumns implements base.LogicalPlan.<2nd> interface.
56+
func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error) {
57+
eCtx := p.SCtx().GetExprCtx().GetEvalCtx()
58+
used := expression.GetUsedList(eCtx, parentUsedCols, p.Schema())
59+
hasBeenUsed := false
60+
for i := range used {
61+
hasBeenUsed = hasBeenUsed || used[i]
62+
if hasBeenUsed {
63+
break
64+
}
65+
}
66+
if !hasBeenUsed {
67+
parentUsedCols = make([]*expression.Column, len(p.Schema().Columns))
68+
copy(parentUsedCols, p.Schema().Columns)
69+
for i := range used {
70+
used[i] = true
71+
}
72+
}
73+
var err error
74+
for i, child := range p.Children() {
75+
p.Children()[i], err = child.PruneColumns(parentUsedCols, opt)
76+
if err != nil {
77+
return nil, err
78+
}
79+
}
80+
81+
prunedColumns := make([]*expression.Column, 0)
82+
for i := len(used) - 1; i >= 0; i-- {
83+
if !used[i] {
84+
prunedColumns = append(prunedColumns, p.Schema().Columns[i])
85+
p.Schema().Columns = append(p.Schema().Columns[:i], p.Schema().Columns[i+1:]...)
86+
}
87+
}
88+
logicaltrace.AppendColumnPruneTraceStep(p, prunedColumns, opt)
89+
if hasBeenUsed {
90+
// It's possible that the child operator adds extra columns to the schema.
91+
// Currently, (*LogicalAggregation).PruneColumns() might do this.
92+
// But we don't need such columns, so we add an extra Projection to prune this column when this happened.
93+
for i, child := range p.Children() {
94+
if p.Schema().Len() < child.Schema().Len() {
95+
schema := p.Schema().Clone()
96+
exprs := make([]expression.Expression, len(p.Schema().Columns))
97+
for j, col := range schema.Columns {
98+
exprs[j] = col
99+
}
100+
proj := LogicalProjection{Exprs: exprs, AvoidColumnEvaluator: true}.Init(p.SCtx(), p.QueryBlockOffset())
101+
proj.SetSchema(schema)
102+
103+
proj.SetChildren(child)
104+
p.Children()[i] = proj
105+
}
106+
}
107+
}
108+
return p, nil
109+
}
110+
111+
// FindBestTask inherits BaseLogicalPlan.LogicalPlan.<3rd> implementation.
112+
113+
// BuildKeyInfo inherits BaseLogicalPlan.LogicalPlan.<4th> implementation.
114+
115+
// PushDownTopN implements the base.LogicalPlan.<5th> interface.
116+
func (p *LogicalUnionAll) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan {
117+
var topN *LogicalTopN
118+
if topNLogicalPlan != nil {
119+
topN = topNLogicalPlan.(*LogicalTopN)
120+
}
121+
for i, child := range p.Children() {
122+
var newTopN *LogicalTopN
123+
if topN != nil {
124+
newTopN = LogicalTopN{Count: topN.Count + topN.Offset, PreferLimitToCop: topN.PreferLimitToCop}.Init(p.SCtx(), topN.QueryBlockOffset())
125+
for _, by := range topN.ByItems {
126+
newTopN.ByItems = append(newTopN.ByItems, &util.ByItems{Expr: by.Expr, Desc: by.Desc})
127+
}
128+
// newTopN to push down Union's child
129+
appendNewTopNTraceStep(topN, p, opt)
130+
}
131+
p.Children()[i] = child.PushDownTopN(newTopN, opt)
132+
}
133+
if topN != nil {
134+
return topN.AttachChild(p, opt)
135+
}
136+
return p
137+
}
138+
139+
// DeriveTopN inherits BaseLogicalPlan.LogicalPlan.<6th> implementation.
140+
141+
// PredicateSimplification inherits BaseLogicalPlan.LogicalPlan.<7th> implementation.
142+
143+
// ConstantPropagation inherits BaseLogicalPlan.LogicalPlan.<8th> implementation.
144+
145+
// PullUpConstantPredicates inherits BaseLogicalPlan.LogicalPlan.<9th> implementation.
146+
147+
// RecursiveDeriveStats inherits BaseLogicalPlan.LogicalPlan.<10th> implementation.
148+
149+
// DeriveStats implement base.LogicalPlan.<11th> interface.
150+
func (p *LogicalUnionAll) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) {
151+
if p.StatsInfo() != nil {
152+
return p.StatsInfo(), nil
153+
}
154+
p.SetStats(&property.StatsInfo{
155+
ColNDVs: make(map[int64]float64, selfSchema.Len()),
156+
})
157+
for _, childProfile := range childStats {
158+
p.StatsInfo().RowCount += childProfile.RowCount
159+
for _, col := range selfSchema.Columns {
160+
p.StatsInfo().ColNDVs[col.UniqueID] += childProfile.ColNDVs[col.UniqueID]
161+
}
162+
}
163+
return p.StatsInfo(), nil
164+
}
165+
166+
// ExtractColGroups inherits BaseLogicalPlan.LogicalPlan.<12th> implementation.
167+
168+
// PreparePossibleProperties implements base.LogicalPlan.<13th> interface.
169+
170+
// ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.
171+
func (p *LogicalUnionAll) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error) {
172+
return exhaustUnionAllPhysicalPlans(p, prop)
173+
}
174+
175+
// ExtractCorrelatedCols inherits BaseLogicalPlan.LogicalPlan.<15th> implementation.
176+
177+
// MaxOneRow inherits BaseLogicalPlan.LogicalPlan.<16th> implementation.
178+
179+
// Children inherits BaseLogicalPlan.LogicalPlan.<17th> implementation.
180+
181+
// SetChildren inherits BaseLogicalPlan.LogicalPlan.<18th> implementation.
182+
183+
// SetChild inherits BaseLogicalPlan.LogicalPlan.<19th> implementation.
184+
185+
// RollBackTaskMap inherits BaseLogicalPlan.LogicalPlan.<20th> implementation.
186+
187+
// CanPushToCop inherits BaseLogicalPlan.LogicalPlan.<21st> implementation.
188+
189+
// ExtractFD inherits BaseLogicalPlan.LogicalPlan.<22nd> implementation.
190+
191+
// GetBaseLogicalPlan inherits BaseLogicalPlan.LogicalPlan.<23rd> implementation.
192+
193+
// ConvertOuterToInnerJoin inherits BaseLogicalPlan.LogicalPlan.<24th> implementation.
194+
195+
// *************************** end implementation of logicalPlan interface ***************************

pkg/planner/core/rule_column_pruning.go

Lines changed: 0 additions & 55 deletions
Original file line numberDiff line numberDiff line change
@@ -140,61 +140,6 @@ func pruneByItems(p base.LogicalPlan, old []*util.ByItems, opt *optimizetrace.Lo
140140
return
141141
}
142142

143-
// PruneColumns implements base.LogicalPlan interface.
144-
func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error) {
145-
used := expression.GetUsedList(p.SCtx().GetExprCtx().GetEvalCtx(), parentUsedCols, p.Schema())
146-
hasBeenUsed := false
147-
for i := range used {
148-
hasBeenUsed = hasBeenUsed || used[i]
149-
if hasBeenUsed {
150-
break
151-
}
152-
}
153-
if !hasBeenUsed {
154-
parentUsedCols = make([]*expression.Column, len(p.Schema().Columns))
155-
copy(parentUsedCols, p.Schema().Columns)
156-
for i := range used {
157-
used[i] = true
158-
}
159-
}
160-
var err error
161-
for i, child := range p.Children() {
162-
p.Children()[i], err = child.PruneColumns(parentUsedCols, opt)
163-
if err != nil {
164-
return nil, err
165-
}
166-
}
167-
168-
prunedColumns := make([]*expression.Column, 0)
169-
for i := len(used) - 1; i >= 0; i-- {
170-
if !used[i] {
171-
prunedColumns = append(prunedColumns, p.Schema().Columns[i])
172-
p.Schema().Columns = append(p.Schema().Columns[:i], p.Schema().Columns[i+1:]...)
173-
}
174-
}
175-
logicaltrace.AppendColumnPruneTraceStep(p, prunedColumns, opt)
176-
if hasBeenUsed {
177-
// It's possible that the child operator adds extra columns to the schema.
178-
// Currently, (*LogicalAggregation).PruneColumns() might do this.
179-
// But we don't need such columns, so we add an extra Projection to prune this column when this happened.
180-
for i, child := range p.Children() {
181-
if p.Schema().Len() < child.Schema().Len() {
182-
schema := p.Schema().Clone()
183-
exprs := make([]expression.Expression, len(p.Schema().Columns))
184-
for j, col := range schema.Columns {
185-
exprs[j] = col
186-
}
187-
proj := LogicalProjection{Exprs: exprs, AvoidColumnEvaluator: true}.Init(p.SCtx(), p.QueryBlockOffset())
188-
proj.SetSchema(schema)
189-
190-
proj.SetChildren(child)
191-
p.Children()[i] = proj
192-
}
193-
}
194-
}
195-
return p, nil
196-
}
197-
198143
// PruneColumns implements base.LogicalPlan interface.
199144
func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error) {
200145
for i := 0; i < p.handleCols.NumCols(); i++ {

pkg/planner/core/rule_predicate_push_down.go

Lines changed: 0 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -420,17 +420,6 @@ func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression
420420
return append(remained, canNotBePushed...), child
421421
}
422422

423-
// PredicatePushDown implements base.LogicalPlan PredicatePushDown interface.
424-
func (p *LogicalUnionAll) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan) {
425-
for i, proj := range p.Children() {
426-
newExprs := make([]expression.Expression, 0, len(predicates))
427-
newExprs = append(newExprs, predicates...)
428-
retCond, newChild := proj.PredicatePushDown(newExprs, opt)
429-
utilfuncp.AddSelection(p, newChild, retCond, i, opt)
430-
}
431-
return nil, p
432-
}
433-
434423
// PredicatePushDown implements base.LogicalPlan PredicatePushDown interface.
435424
func (p *LogicalMaxOneRow) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan) {
436425
// MaxOneRow forbids any condition to push down.

pkg/planner/core/rule_topn_push_down.go

Lines changed: 0 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -65,30 +65,6 @@ func (p *LogicalCTE) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimiz
6565
return p
6666
}
6767

68-
// PushDownTopN implements the LogicalPlan interface.
69-
func (p *LogicalUnionAll) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan {
70-
var topN *LogicalTopN
71-
if topNLogicalPlan != nil {
72-
topN = topNLogicalPlan.(*LogicalTopN)
73-
}
74-
for i, child := range p.Children() {
75-
var newTopN *LogicalTopN
76-
if topN != nil {
77-
newTopN = LogicalTopN{Count: topN.Count + topN.Offset, PreferLimitToCop: topN.PreferLimitToCop}.Init(p.SCtx(), topN.QueryBlockOffset())
78-
for _, by := range topN.ByItems {
79-
newTopN.ByItems = append(newTopN.ByItems, &util.ByItems{Expr: by.Expr, Desc: by.Desc})
80-
}
81-
// newTopN to push down Union's child
82-
appendNewTopNTraceStep(topN, p, opt)
83-
}
84-
p.Children()[i] = child.PushDownTopN(newTopN, opt)
85-
}
86-
if topN != nil {
87-
return topN.AttachChild(p, opt)
88-
}
89-
return p
90-
}
91-
9268
// PushDownTopN implements LogicalPlan interface.
9369
func (p *LogicalProjection) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan {
9470
var topN *LogicalTopN

pkg/planner/core/stats.go

Lines changed: 0 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -555,23 +555,6 @@ func (p *LogicalSelection) DeriveStats(childStats []*property.StatsInfo, _ *expr
555555
return p.StatsInfo(), nil
556556
}
557557

558-
// DeriveStats implement LogicalPlan DeriveStats interface.
559-
func (p *LogicalUnionAll) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) {
560-
if p.StatsInfo() != nil {
561-
return p.StatsInfo(), nil
562-
}
563-
p.SetStats(&property.StatsInfo{
564-
ColNDVs: make(map[int64]float64, selfSchema.Len()),
565-
})
566-
for _, childProfile := range childStats {
567-
p.StatsInfo().RowCount += childProfile.RowCount
568-
for _, col := range selfSchema.Columns {
569-
p.StatsInfo().ColNDVs[col.UniqueID] += childProfile.ColNDVs[col.UniqueID]
570-
}
571-
}
572-
return p.StatsInfo(), nil
573-
}
574-
575558
func deriveLimitStats(childProfile *property.StatsInfo, limitCount float64) *property.StatsInfo {
576559
stats := &property.StatsInfo{
577560
RowCount: math.Min(limitCount, childProfile.RowCount),

0 commit comments

Comments
 (0)