Skip to content

Commit 240702e

Browse files
authored
planner: move base physical plan into physicalop pkg. (#55131)
ref #51664, ref #52714
1 parent 876268f commit 240702e

31 files changed

+1017
-899
lines changed

pkg/planner/core/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -138,6 +138,7 @@ go_library(
138138
"//pkg/planner/core/metrics",
139139
"//pkg/planner/core/operator/baseimpl",
140140
"//pkg/planner/core/operator/logicalop",
141+
"//pkg/planner/core/operator/physicalop",
141142
"//pkg/planner/core/rule",
142143
"//pkg/planner/core/rule/util",
143144
"//pkg/planner/funcdep",
@@ -290,6 +291,7 @@ go_test(
290291
"//pkg/planner",
291292
"//pkg/planner/core/base",
292293
"//pkg/planner/core/operator/logicalop",
294+
"//pkg/planner/core/operator/physicalop",
293295
"//pkg/planner/property",
294296
"//pkg/planner/util",
295297
"//pkg/planner/util/coretestsdk",

pkg/planner/core/common_plans.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ import (
2828
"github.com/pingcap/tidb/pkg/parser/model"
2929
"github.com/pingcap/tidb/pkg/parser/mysql"
3030
"github.com/pingcap/tidb/pkg/planner/core/base"
31+
"github.com/pingcap/tidb/pkg/planner/core/operator/physicalop"
3132
"github.com/pingcap/tidb/pkg/planner/property"
3233
"github.com/pingcap/tidb/pkg/planner/util"
3334
"github.com/pingcap/tidb/pkg/planner/util/costusage"
@@ -307,7 +308,7 @@ func (s *Simple) MemoryUsage() (sum int64) {
307308
//
308309
// Used for simple statements executing in coprocessor.
309310
type PhysicalSimpleWrapper struct {
310-
basePhysicalPlan
311+
physicalop.BasePhysicalPlan
311312
Inner Simple
312313
}
313314

@@ -317,7 +318,7 @@ func (p *PhysicalSimpleWrapper) MemoryUsage() (sum int64) {
317318
return
318319
}
319320

320-
sum = p.basePhysicalPlan.MemoryUsage() + p.Inner.MemoryUsage()
321+
sum = p.BasePhysicalPlan.MemoryUsage() + p.Inner.MemoryUsage()
321322
return
322323
}
323324

pkg/planner/core/core_init.go

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ func init() {
3535
utilfuncp.GetStreamAggs = getStreamAggs
3636
utilfuncp.GetHashAggs = getHashAggs
3737
utilfuncp.PruneByItems = pruneByItems
38+
utilfuncp.PushDownTopNForBaseLogicalPlan = pushDownTopNForBaseLogicalPlan
3839
utilfuncp.FindBestTask4LogicalShow = findBestTask4LogicalShow
3940
utilfuncp.FindBestTask4LogicalCTETable = findBestTask4LogicalCTETable
4041
utilfuncp.FindBestTask4LogicalMemTable = findBestTask4LogicalMemTable
@@ -50,8 +51,11 @@ func init() {
5051
utilfuncp.ExhaustPhysicalPlans4LogicalUnionScan = exhaustPhysicalPlans4LogicalUnionScan
5152
utilfuncp.ExhaustPhysicalPlans4LogicalProjection = exhaustPhysicalPlans4LogicalProjection
5253

54+
utilfuncp.GetActualProbeCntFromProbeParents = getActualProbeCntFromProbeParents
55+
utilfuncp.GetEstimatedProbeCntFromProbeParents = getEstimatedProbeCntFromProbeParents
5356
utilfuncp.AppendCandidate4PhysicalOptimizeOp = appendCandidate4PhysicalOptimizeOp
54-
utilfuncp.PushDownTopNForBaseLogicalPlan = pushDownTopNForBaseLogicalPlan
57+
58+
utilfuncp.AttachPlan2Task = attachPlan2Task
5559

5660
// For mv index init.
5761
cardinality.GetTblInfoForUsedStatsByPhysicalID = getTblInfoForUsedStatsByPhysicalID

pkg/planner/core/exhaust_physical_plans.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -210,7 +210,7 @@ func GetMergeJoin(p *LogicalJoin, prop *property.PhysicalProperty, schema *expre
210210
reqProps[0].ExpectedCnt = leftStatsInfo.RowCount * expCntScale
211211
reqProps[1].ExpectedCnt = rightStatsInfo.RowCount * expCntScale
212212
}
213-
mergeJoin.childrenReqProps = reqProps
213+
mergeJoin.SetChildrenReqProps(reqProps)
214214
_, desc := prop.AllSameOrder()
215215
mergeJoin.Desc = desc
216216
joins = append(joins, mergeJoin)
@@ -356,7 +356,7 @@ func getEnforcedMergeJoin(p *LogicalJoin, prop *property.PhysicalProperty, schem
356356
}
357357
enforcedPhysicalMergeJoin := PhysicalMergeJoin{basePhysicalJoin: baseJoin, Desc: desc}.Init(p.SCtx(), statsInfo.ScaleByExpectCnt(prop.ExpectedCnt), p.QueryBlockOffset())
358358
enforcedPhysicalMergeJoin.SetSchema(schema)
359-
enforcedPhysicalMergeJoin.childrenReqProps = []*property.PhysicalProperty{lProp, rProp}
359+
enforcedPhysicalMergeJoin.SetChildrenReqProps([]*property.PhysicalProperty{lProp, rProp})
360360
enforcedPhysicalMergeJoin.initCompareFuncs()
361361
return []base.PhysicalPlan{enforcedPhysicalMergeJoin}
362362
}
@@ -2965,7 +2965,7 @@ func exhaustPhysicalPlans4LogicalCTE(p *LogicalCTE, prop *property.PhysicalPrope
29652965
}.Init(p.SCtx(), p.StatsInfo())
29662966
}
29672967
pcte.SetSchema(p.Schema())
2968-
pcte.childrenReqProps = []*property.PhysicalProperty{prop.CloneEssentialFields()}
2968+
pcte.SetChildrenReqProps([]*property.PhysicalProperty{prop.CloneEssentialFields()})
29692969
return []base.PhysicalPlan{(*PhysicalCTEStorage)(pcte)}, true, nil
29702970
}
29712971

pkg/planner/core/find_best_task_test.go

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import (
2222
"github.com/pingcap/tidb/pkg/expression"
2323
"github.com/pingcap/tidb/pkg/planner/core/base"
2424
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
25+
"github.com/pingcap/tidb/pkg/planner/core/operator/physicalop"
2526
"github.com/pingcap/tidb/pkg/planner/property"
2627
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
2728
"github.com/stretchr/testify/require"
@@ -76,16 +77,16 @@ func (p mockLogicalPlan4Test) Init(ctx base.PlanContext) *mockLogicalPlan4Test {
7677
func (p *mockLogicalPlan4Test) getPhysicalPlan1(prop *property.PhysicalProperty) base.PhysicalPlan {
7778
physicalPlan1 := mockPhysicalPlan4Test{planType: 1}.Init(p.SCtx())
7879
physicalPlan1.SetStats(&property.StatsInfo{RowCount: 1})
79-
physicalPlan1.childrenReqProps = make([]*property.PhysicalProperty, 1)
80-
physicalPlan1.childrenReqProps[0] = prop.CloneEssentialFields()
80+
physicalPlan1.SetChildrenReqProps(make([]*property.PhysicalProperty, 1))
81+
physicalPlan1.SetXthChildReqProps(0, prop.CloneEssentialFields())
8182
return physicalPlan1
8283
}
8384

8485
func (p *mockLogicalPlan4Test) getPhysicalPlan2(prop *property.PhysicalProperty) base.PhysicalPlan {
8586
physicalPlan2 := mockPhysicalPlan4Test{planType: 2}.Init(p.SCtx())
8687
physicalPlan2.SetStats(&property.StatsInfo{RowCount: 1})
87-
physicalPlan2.childrenReqProps = make([]*property.PhysicalProperty, 1)
88-
physicalPlan2.childrenReqProps[0] = property.NewPhysicalProperty(prop.TaskTp, nil, false, prop.ExpectedCnt, false)
88+
physicalPlan2.SetChildrenReqProps(make([]*property.PhysicalProperty, 1))
89+
physicalPlan2.SetXthChildReqProps(0, property.NewPhysicalProperty(prop.TaskTp, nil, false, prop.ExpectedCnt, false))
8990
return physicalPlan2
9091
}
9192

@@ -115,14 +116,14 @@ func (p *mockLogicalPlan4Test) ExhaustPhysicalPlans(prop *property.PhysicalPrope
115116
}
116117

117118
type mockPhysicalPlan4Test struct {
118-
basePhysicalPlan
119+
physicalop.BasePhysicalPlan
119120
// 1 or 2 for physicalPlan1 or physicalPlan2.
120121
// See the comment of mockLogicalPlan4Test.
121122
planType int
122123
}
123124

124125
func (p mockPhysicalPlan4Test) Init(ctx base.PlanContext) *mockPhysicalPlan4Test {
125-
p.basePhysicalPlan = newBasePhysicalPlan(ctx, "mockPlan", &p, 0)
126+
p.BasePhysicalPlan = physicalop.NewBasePhysicalPlan(ctx, "mockPlan", &p, 0)
126127
return &p
127128
}
128129

pkg/planner/core/flat_plan.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -372,7 +372,7 @@ func (f *FlatPhysicalPlan) flattenRecursively(p base.Plan, info *operatorCtx, ta
372372
// We shallow copy the PhysicalCTE here because we don't want the probeParents (see comments in PhysicalPlan
373373
// for details) to affect the row count display of the independent CTE plan tree.
374374
copiedCTE := *plan
375-
copiedCTE.probeParents = nil
375+
copiedCTE.SetProbeParents(nil)
376376
if info.isRoot {
377377
// If it's executed in TiDB, we need to record it since we don't have producer and consumer
378378
f.ctesToFlatten = append(f.ctesToFlatten, &copiedCTE)

pkg/planner/core/foreign_key.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -24,13 +24,14 @@ import (
2424
"github.com/pingcap/tidb/pkg/parser/model"
2525
"github.com/pingcap/tidb/pkg/parser/mysql"
2626
"github.com/pingcap/tidb/pkg/planner/core/base"
27+
"github.com/pingcap/tidb/pkg/planner/core/operator/physicalop"
2728
"github.com/pingcap/tidb/pkg/table"
2829
"github.com/pingcap/tidb/pkg/util/dbterror/plannererrors"
2930
)
3031

3132
// FKCheck indicates the foreign key constraint checker.
3233
type FKCheck struct {
33-
basePhysicalPlan
34+
physicalop.BasePhysicalPlan
3435
FK *model.FKInfo
3536
ReferredFK *model.ReferredFKInfo
3637
Tbl table.Table
@@ -46,7 +47,7 @@ type FKCheck struct {
4647

4748
// FKCascade indicates the foreign key constraint cascade behaviour.
4849
type FKCascade struct {
49-
basePhysicalPlan
50+
physicalop.BasePhysicalPlan
5051
Tp FKCascadeType
5152
ReferredFK *model.ReferredFKInfo
5253
ChildTable table.Table

pkg/planner/core/fragment.go

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -229,7 +229,7 @@ func (f *Fragment) init(p base.PhysicalPlan) error {
229229
f.TableScan = x
230230
case *PhysicalExchangeReceiver:
231231
// TODO: after we support partial merge, we should check whether all the target exchangeReceiver is same.
232-
f.singleton = f.singleton || x.children[0].(*PhysicalExchangeSender).ExchangeType == tipb.ExchangeType_PassThrough
232+
f.singleton = f.singleton || x.Children()[0].(*PhysicalExchangeSender).ExchangeType == tipb.ExchangeType_PassThrough
233233
f.ExchangeReceivers = append(f.ExchangeReceivers, x)
234234
case *PhysicalUnionAll:
235235
return errors.New("unexpected union all detected")
@@ -284,12 +284,12 @@ func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []base.PhysicalPla
284284
e.CTEGroups[cte.CTE.IDForStorage].CTEReader = append(e.CTEGroups[cte.CTE.IDForStorage].CTEReader, cte)
285285
}
286286
case *PhysicalHashJoin:
287-
stack = append(stack, x.children[1-x.InnerChildIdx])
287+
stack = append(stack, x.Children()[1-x.InnerChildIdx])
288288
err := e.untwistPlanAndRemoveUnionAll(stack, forest)
289289
stack = stack[:len(stack)-1]
290290
return errors.Trace(err)
291291
case *PhysicalUnionAll:
292-
for _, ch := range x.children {
292+
for _, ch := range x.Children() {
293293
stack = append(stack, ch)
294294
err := e.untwistPlanAndRemoveUnionAll(stack, forest)
295295
stack = stack[:len(stack)-1]
@@ -298,19 +298,19 @@ func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []base.PhysicalPla
298298
}
299299
}
300300
case *PhysicalSequence:
301-
lastChildIdx := len(x.children) - 1
301+
lastChildIdx := len(x.Children()) - 1
302302
// except the last child, those previous ones are all cte producer.
303303
for i := 0; i < lastChildIdx; i++ {
304304
if e.CTEGroups == nil {
305305
e.CTEGroups = make(map[int]*cteGroupInFragment)
306306
}
307-
cteStorage := x.children[i].(*PhysicalCTEStorage)
307+
cteStorage := x.Children()[i].(*PhysicalCTEStorage)
308308
e.CTEGroups[cteStorage.CTE.IDForStorage] = &cteGroupInFragment{
309309
CTEStorage: cteStorage,
310310
CTEReader: make([]*PhysicalCTE, 0, 3),
311311
}
312312
}
313-
stack = append(stack, x.children[lastChildIdx])
313+
stack = append(stack, x.Children()[lastChildIdx])
314314
err := e.untwistPlanAndRemoveUnionAll(stack, forest)
315315
stack = stack[:len(stack)-1]
316316
if err != nil {
@@ -397,7 +397,7 @@ func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv
397397
}
398398
cteProducerTasks := make([]*kv.MPPTask, 0)
399399
for _, cteR := range f.CTEReaders {
400-
child := cteR.children[0]
400+
child := cteR.Children()[0]
401401
if _, ok := child.(*PhysicalProjection); ok {
402402
child = child.Children()[0]
403403
}
@@ -450,7 +450,7 @@ func (f *Fragment) flipCTEReader(currentPlan base.PhysicalPlan) {
450450
func (e *mppTaskGenerator) generateTasksForCTEReader(cteReader *PhysicalCTE) (err error) {
451451
group := e.CTEGroups[cteReader.CTE.IDForStorage]
452452
if group.StorageFragments == nil {
453-
group.CTEStorage.storageSender.SetChildren(group.CTEStorage.children...)
453+
group.CTEStorage.storageSender.SetChildren(group.CTEStorage.Children()...)
454454
group.StorageTasks, group.StorageFragments, err = e.generateMPPTasksForExchangeSender(group.CTEStorage.storageSender)
455455
if err != nil {
456456
return err
@@ -460,16 +460,16 @@ func (e *mppTaskGenerator) generateTasksForCTEReader(cteReader *PhysicalCTE) (er
460460
receiver.Tasks = group.StorageTasks
461461
receiver.frags = group.StorageFragments
462462
cteReader.SetChildren(receiver)
463-
receiver.SetChildren(group.CTEStorage.children[0])
463+
receiver.SetChildren(group.CTEStorage.Children()[0])
464464
inconsistenceNullable := false
465465
for i, col := range cteReader.schema.Columns {
466-
if mysql.HasNotNullFlag(col.RetType.GetFlag()) != mysql.HasNotNullFlag(group.CTEStorage.children[0].Schema().Columns[i].RetType.GetFlag()) {
466+
if mysql.HasNotNullFlag(col.RetType.GetFlag()) != mysql.HasNotNullFlag(group.CTEStorage.Children()[0].Schema().Columns[i].RetType.GetFlag()) {
467467
inconsistenceNullable = true
468468
break
469469
}
470470
}
471471
if inconsistenceNullable {
472-
cols := group.CTEStorage.children[0].Schema().Clone().Columns
472+
cols := group.CTEStorage.Children()[0].Schema().Clone().Columns
473473
for i, col := range cols {
474474
col.Index = i
475475
}

pkg/planner/core/hint_utils.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -184,28 +184,28 @@ func genHintsFromSingle(p base.PhysicalPlan, nodeType h.NodeType, storeType kv.S
184184
})
185185
}
186186
case *PhysicalMergeJoin:
187-
hint := genJoinMethodHintForSinglePhysicalJoin(p.SCtx(), h.HintSMJ, p.QueryBlockOffset(), nodeType, pp.children...)
187+
hint := genJoinMethodHintForSinglePhysicalJoin(p.SCtx(), h.HintSMJ, p.QueryBlockOffset(), nodeType, pp.Children()...)
188188
if hint != nil {
189189
res = append(res, hint)
190190
}
191191
case *PhysicalHashJoin:
192192
// TODO: support the hash_join_build and hash_join_probe hint for auto capture
193-
hint := genJoinMethodHintForSinglePhysicalJoin(p.SCtx(), h.HintHJ, p.QueryBlockOffset(), nodeType, pp.children...)
193+
hint := genJoinMethodHintForSinglePhysicalJoin(p.SCtx(), h.HintHJ, p.QueryBlockOffset(), nodeType, pp.Children()...)
194194
if hint != nil {
195195
res = append(res, hint)
196196
}
197197
case *PhysicalIndexJoin:
198-
hint := genJoinMethodHintForSinglePhysicalJoin(p.SCtx(), h.HintINLJ, p.QueryBlockOffset(), nodeType, pp.children[pp.InnerChildIdx])
198+
hint := genJoinMethodHintForSinglePhysicalJoin(p.SCtx(), h.HintINLJ, p.QueryBlockOffset(), nodeType, pp.Children()[pp.InnerChildIdx])
199199
if hint != nil {
200200
res = append(res, hint)
201201
}
202202
case *PhysicalIndexMergeJoin:
203-
hint := genJoinMethodHintForSinglePhysicalJoin(p.SCtx(), h.HintINLMJ, p.QueryBlockOffset(), nodeType, pp.children[pp.InnerChildIdx])
203+
hint := genJoinMethodHintForSinglePhysicalJoin(p.SCtx(), h.HintINLMJ, p.QueryBlockOffset(), nodeType, pp.Children()[pp.InnerChildIdx])
204204
if hint != nil {
205205
res = append(res, hint)
206206
}
207207
case *PhysicalIndexHashJoin:
208-
hint := genJoinMethodHintForSinglePhysicalJoin(p.SCtx(), h.HintINLHJ, p.QueryBlockOffset(), nodeType, pp.children[pp.InnerChildIdx])
208+
hint := genJoinMethodHintForSinglePhysicalJoin(p.SCtx(), h.HintINLHJ, p.QueryBlockOffset(), nodeType, pp.Children()[pp.InnerChildIdx])
209209
if hint != nil {
210210
res = append(res, hint)
211211
}

0 commit comments

Comments
 (0)