From 3f91dd30ca94eed520e493b2d28cfc39bbc62161 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 15 Feb 2023 12:23:57 +0000 Subject: [PATCH 01/24] tmp --- planner/core/access_object.go | 6 +- planner/core/exhaust_physical_plans.go | 66 ++++- planner/core/find_best_task.go | 229 +++++++++++++++++- planner/core/hints.go | 5 +- planner/core/initialize.go | 14 +- planner/core/logical_plan_builder.go | 55 ++++- planner/core/logical_plans.go | 10 + planner/core/optimizer.go | 32 ++- planner/core/physical_plans.go | 61 +++++ planner/core/planbuilder.go | 5 + planner/core/rule_column_pruning.go | 10 + planner/core/rule_decorrelate.go | 4 + planner/core/rule_eliminate_projection.go | 4 + .../core/rule_generate_column_substitute.go | 3 + planner/core/rule_join_elimination.go | 4 + planner/core/rule_join_reorder.go | 4 + planner/core/rule_max_min_eliminate.go | 4 + planner/core/rule_partition_processor.go | 2 + planner/core/rule_predicate_push_down.go | 7 + planner/core/rule_semi_join_rewrite.go | 3 + planner/core/rule_topn_push_down.go | 7 + planner/core/stats.go | 35 ++- planner/core/stringer.go | 8 + planner/core/task.go | 45 ++++ planner/property/physical_property.go | 8 + sessionctx/variable/session.go | 3 + util/plancodec/id.go | 2 + 27 files changed, 598 insertions(+), 38 deletions(-) diff --git a/planner/core/access_object.go b/planner/core/access_object.go index c9994efe3de86..40c938a83e5d0 100644 --- a/planner/core/access_object.go +++ b/planner/core/access_object.go @@ -396,7 +396,11 @@ func (p *PhysicalTableReader) accessObject(sctx sessionctx.Context) AccessObject return DynamicPartitionAccessObjects(nil) } if len(p.PartitionInfos) == 0 { - ts := p.TablePlans[0].(*PhysicalTableScan) + ts, ok := p.TablePlans[0].(*PhysicalTableScan) + if !ok { + cte := p.TablePlans[0].(*PhysicalCTE) + return OtherAccessObject(fmt.Sprintf("cte: %v as %v", cte.cteName, cte.cteAsName)) + } asName := "" if ts.TableAsName != nil && len(ts.TableAsName.O) > 0 { asName = ts.TableAsName.O diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 205a2d8242b4b..0c087c8f780dc 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2117,7 +2117,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC baseJoin.InnerChildIdx = preferredBuildIndex childrenProps := make([]*property.PhysicalProperty, 2) if useBCJ { - childrenProps[preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.BroadcastType, CanAddEnforcer: true, RejectSort: true} + childrenProps[preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.BroadcastType, CanAddEnforcer: true, RejectSort: true, CTECanMPP: prop.CTECanMPP} expCnt := math.MaxFloat64 if prop.ExpectedCnt < p.stats.RowCount { expCntScale := prop.ExpectedCnt / p.stats.RowCount @@ -2130,12 +2130,12 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC hashKeys = lPartitionKeys } if matches := prop.IsSubsetOf(hashKeys); len(matches) != 0 { - childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.HashType, MPPPartitionCols: prop.MPPPartitionCols, RejectSort: true} + childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.HashType, MPPPartitionCols: prop.MPPPartitionCols, RejectSort: true, CTECanMPP: prop.CTECanMPP} } else { return nil } } else { - childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.AnyType, RejectSort: true} + childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.AnyType, RejectSort: true, CTECanMPP: prop.CTECanMPP} } } else { lPartitionKeys, rPartitionKeys := p.GetPotentialPartitionKeys() @@ -2269,7 +2269,7 @@ func pushLimitOrTopNForcibly(p LogicalPlan) bool { return false } -func (lt *LogicalTopN) getPhysTopN(_ *property.PhysicalProperty) []PhysicalPlan { +func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPlan { allTaskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType} if !pushLimitOrTopNForcibly(lt) { allTaskTypes = append(allTaskTypes, property.RootTaskType) @@ -2279,7 +2279,7 @@ func (lt *LogicalTopN) getPhysTopN(_ *property.PhysicalProperty) []PhysicalPlan } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { - resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: math.MaxFloat64} + resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: math.MaxFloat64, CTECanMPP: prop.CTECanMPP} topN := PhysicalTopN{ ByItems: lt.ByItems, Count: lt.Count, @@ -2454,6 +2454,7 @@ func (lw *LogicalWindow) tryToGetMppWindows(prop *property.PhysicalProperty) []P SortItems: byItems, TaskTp: property.MppTaskType, SortItemsForPartition: byItems, + CTECanMPP: prop.CTECanMPP, } if !prop.IsPrefix(childProperty) { return nil @@ -2593,6 +2594,24 @@ func (p *baseLogicalPlan) canPushToCopImpl(storeTp kv.StoreType, considerDual bo // These operators can be partially push down to TiFlash, so we don't raise warning for them. case *LogicalLimit, *LogicalTopN: return false + case *LogicalSequence: + if storeTp == kv.TiFlash { + for _, cte := range c.ctes { + if cte.cteClass.recursivePartLogicalPlan != nil || !cte.cteClass.seedPartLogicalPlan.canPushToCop(storeTp) { + return false + } + } + return true + } + return false + case *LogicalCTE: + if storeTp != kv.TiFlash { + return false + } + if c.cte.recursivePartLogicalPlan != nil || !c.cte.seedPartLogicalPlan.canPushToCop(storeTp) { + return false + } + return true default: p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced( "MPP mode may be blocked because operator `" + c.TP() + "` is not supported now.") @@ -2788,7 +2807,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // To avoid mess, we don't do any one-phase aggregation in this case. // If this is a skew distinct group agg, skip generating 1-phase agg, because skew data will cause performance issue if len(partitionCols) != 0 && !la.ctx.GetSessionVars().EnableSkewDistinctAgg { - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: partitionCols, CanAddEnforcer: true, RejectSort: true} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: partitionCols, CanAddEnforcer: true, RejectSort: true, CTECanMPP: prop.CTECanMPP} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = Mpp1Phase @@ -3004,7 +3023,7 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { - resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(p.Count + p.Offset)} + resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(p.Count + p.Offset), CTECanMPP: prop.CTECanMPP} limit := PhysicalLimit{ Offset: p.Offset, Count: p.Count, @@ -3047,6 +3066,7 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ExpectedCnt: prop.ExpectedCnt, TaskTp: property.MppTaskType, RejectSort: true, + CTECanMPP: prop.CTECanMPP, }) } else { chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: prop.ExpectedCnt, RejectSort: true}) @@ -3084,7 +3104,7 @@ func (p *LogicalPartitionUnionAll) exhaustPhysicalPlans(prop *property.PhysicalP } func (ls *LogicalSort) getPhysicalSort(prop *property.PhysicalProperty) *PhysicalSort { - ps := PhysicalSort{ByItems: ls.ByItems}.Init(ls.ctx, ls.stats.ScaleByExpectCnt(prop.ExpectedCnt), ls.blockOffset, &property.PhysicalProperty{TaskTp: prop.TaskTp, ExpectedCnt: math.MaxFloat64, RejectSort: true}) + ps := PhysicalSort{ByItems: ls.ByItems}.Init(ls.ctx, ls.stats.ScaleByExpectCnt(prop.ExpectedCnt), ls.blockOffset, &property.PhysicalProperty{TaskTp: prop.TaskTp, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTECanMPP: prop.CTECanMPP}) return ps } @@ -3131,3 +3151,33 @@ func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) mor := PhysicalMaxOneRow{}.Init(p.ctx, p.stats, p.blockOffset, &property.PhysicalProperty{ExpectedCnt: 2}) return []PhysicalPlan{mor}, true, nil } + +func (p *LogicalCTE) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { + pcte := PhysicalCTE{CTE: p.cte}.Init(p.ctx, p.stats) + pcte.childrenReqProps = []*property.PhysicalProperty{prop.CloneEssentialFields()} + return []PhysicalPlan{(*PhysicalCTEStorage)(pcte)}, true, nil +} + +func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { + possibleChildrenProps := make([]*property.PhysicalProperty, 0, 2) + childProp := prop.CloneEssentialFields() + possibleChildrenProps = append(possibleChildrenProps, childProp) + + if prop.TaskTp != property.MppTaskType && + p.SCtx().GetSessionVars().IsMPPAllowed() { + childPropMpp := prop.CloneEssentialFields() + childPropMpp.TaskTp = property.MppTaskType + possibleChildrenProps = append(possibleChildrenProps, childPropMpp) + } + seqs := make([]PhysicalPlan, 0, 2) + for _, propChoice := range possibleChildrenProps { + childReqs := make([]*property.PhysicalProperty, 0, len(p.children)) + for i := 0; i < len(p.children)-1; i++ { + childReqs = append(childReqs, propChoice) + } + childReqs = append(childReqs, propChoice) + seq := PhysicalSequence{}.Init(p.ctx, p.stats, p.blockOffset, childReqs...) + seqs = append(seqs, seq) + } + return seqs, true, nil +} diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index c2d3e1a62d7bc..15b97517fde20 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -248,6 +248,115 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } } + if _, ok := pp.(*PhysicalCTEStorage); ok { + logutil.BgLogger().Warn("xxxxx") + } + // Combine best child tasks with parent physical plan. + curTask := pp.attach2Task(childTasks...) + + if curTask.invalid() { + continue + } + + // An optimal task could not satisfy the property, so it should be converted here. + if _, ok := curTask.(*rootTask); !ok && prop.TaskTp == property.RootTaskType { + curTask = curTask.convertToRootTask(p.ctx) + } + + // Enforce curTask property + if addEnforcer { + curTask = enforceProperty(prop, curTask, p.basePlan.ctx) + } + + // Optimize by shuffle executor to running in parallel manner. + if _, isMpp := curTask.(*mppTask); !isMpp && prop.IsSortItemEmpty() { + // Currently, we do not regard shuffled plan as a new plan. + curTask = optimizeByShuffle(curTask, p.basePlan.ctx) + } + + cntPlan += curCntPlan + planCounter.Dec(curCntPlan) + + if planCounter.Empty() { + bestTask = curTask + break + } + opt.appendCandidate(p, curTask.plan(), prop) + // Get the most efficient one. + if curIsBetter, err := compareTaskCost(p.ctx, curTask, bestTask, opt); err != nil { + return nil, 0, err + } else if curIsBetter { + bestTask = curTask + } + } + return bestTask, cntPlan, nil +} + +func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, + prop *property.PhysicalProperty, addEnforcer bool, planCounter *PlanCounterTp, opt *physicalOptimizeOp) (task, int64, error) { + var bestTask task = invalidTask + var curCntPlan, cntPlan int64 + childTasks := make([]task, 0, len(p.children)) + childCnts := make([]int64, len(p.children)) + cntPlan = 0 + origCTECanMPP := p.ctx.GetSessionVars().CurrentCTECanMPP + defer func() { + p.ctx.GetSessionVars().CurrentCTECanMPP = origCTECanMPP + }() + for _, pp := range physicalPlans { + // Find best child tasks firstly. + childTasks = childTasks[:0] + // The curCntPlan records the number of possible plans for pp + curCntPlan = 1 + timeStampNow := p.GetLogicalTS4TaskMap() + savedPlanID := p.ctx.GetSessionVars().PlanID + allMpp := true + lastIdx := len(p.children) - 1 + p.ctx.GetSessionVars().CurrentCTECanMPP = true + for j := 0; j < lastIdx; j++ { + child := p.children[j] + childProp := pp.GetChildReqProps(j) + childTask, cnt, err := child.findBestTask(childProp, &PlanCounterDisabled, opt) + childCnts[j] = cnt + if err != nil { + return nil, 0, err + } + curCntPlan = curCntPlan * cnt + if childTask != nil && childTask.invalid() { + break + } + childTasks = append(childTasks, childTask) + _, isMpp := childTask.(*mppTask) + allMpp = allMpp && isMpp + } + lastChildProp := pp.GetChildReqProps(lastIdx).CloneEssentialFields() + lastChildProp.CTECanMPP = allMpp + lastChildTask, cnt, err := p.Children()[lastIdx].findBestTask(lastChildProp, &PlanCounterDisabled, opt) + childCnts[lastIdx] = cnt + if err != nil { + return nil, 0, err + } + curCntPlan = curCntPlan * cnt + if lastChildTask != nil && lastChildTask.invalid() { + break + } + childTasks = append(childTasks, lastChildTask) + + // This check makes sure that there is no invalid child task. + if len(childTasks) != len(p.children) { + continue + } + + // If the target plan can be found in this physicalPlan(pp), rebuild childTasks to build the corresponding combination. + if planCounter.IsForce() && int64(*planCounter) <= curCntPlan { + p.ctx.GetSessionVars().PlanID = savedPlanID + curCntPlan = int64(*planCounter) + err := p.rebuildChildTasks(&childTasks, pp, childCnts, int64(*planCounter), timeStampNow, opt) + if err != nil { + return nil, 0, err + } + } + // Combine best child tasks with parent physical plan. curTask := pp.attach2Task(childTasks...) @@ -466,6 +575,107 @@ END: return bestTask, cntPlan, nil } +// findBestTask implements LogicalPlan interface. +func (p *LogicalSequence) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *physicalOptimizeOp) (bestTask task, cntPlan int64, err error) { + // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, + // and set inner child prop nil, so here we do nothing. + if prop == nil { + return nil, 1, nil + } + // Look up the task with this prop in the task map. + // It's used to reduce double counting. + bestTask = p.getTask(prop) + if bestTask != nil { + planCounter.Dec(1) + return bestTask, 1, nil + } + + canAddEnforcer := prop.CanAddEnforcer + + if prop.TaskTp != property.RootTaskType && !prop.IsFlashProp() { + // Currently all plan cannot totally push down to TiKV. + p.storeTask(prop, invalidTask) + return invalidTask, 0, nil + } + + bestTask = invalidTask + cntPlan = 0 + // prop should be read only because its cached hashcode might be not consistent + // when it is changed. So we clone a new one for the temporary changes. + newProp := prop.CloneEssentialFields() + var plansFitsProp, plansNeedEnforce []PhysicalPlan + var hintWorksWithProp bool + // Maybe the plan can satisfy the required property, + // so we try to get the task without the enforced sort first. + plansFitsProp, hintWorksWithProp, err = p.self.exhaustPhysicalPlans(newProp) + if err != nil { + return nil, 0, err + } + if !hintWorksWithProp && !newProp.IsSortItemEmpty() { + // If there is a hint in the plan and the hint cannot satisfy the property, + // we enforce this property and try to generate the PhysicalPlan again to + // make sure the hint can work. + canAddEnforcer = true + } + + if canAddEnforcer { + // Then, we use the empty property to get physicalPlans and + // try to get the task with an enforced sort. + newProp.SortItems = []property.SortItem{} + newProp.SortItemsForPartition = []property.SortItem{} + newProp.ExpectedCnt = math.MaxFloat64 + newProp.MPPPartitionCols = nil + newProp.MPPPartitionTp = property.AnyType + var hintCanWork bool + plansNeedEnforce, hintCanWork, err = p.self.exhaustPhysicalPlans(newProp) + if err != nil { + return nil, 0, err + } + if hintCanWork && !hintWorksWithProp { + // If the hint can work with the empty property, but cannot work with + // the required property, we give up `plansFitProp` to make sure the hint + // can work. + plansFitsProp = nil + } + if !hintCanWork && !hintWorksWithProp && !prop.CanAddEnforcer { + // If the original property is not enforced and hint cannot + // work anyway, we give up `plansNeedEnforce` for efficiency, + plansNeedEnforce = nil + } + newProp = prop + } + + var cnt int64 + var curTask task + if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp, false, planCounter, opt); err != nil { + return nil, 0, err + } + cntPlan += cnt + if planCounter.Empty() { + goto END + } + + curTask, cnt, err = p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp, true, planCounter, opt) + if err != nil { + return nil, 0, err + } + cntPlan += cnt + if planCounter.Empty() { + bestTask = curTask + goto END + } + opt.appendCandidate(p, curTask.plan(), prop) + if curIsBetter, err := compareTaskCost(p.ctx, curTask, bestTask, opt); err != nil { + return nil, 0, err + } else if curIsBetter { + bestTask = curTask + } + +END: + p.storeTask(prop, bestTask) + return bestTask, cntPlan, nil +} + func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *physicalOptimizeOp) (t task, cntPlan int64, err error) { if prop.MPPPartitionTp != property.AnyType { return invalidTask, 0, nil @@ -2419,14 +2629,29 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper return is } -func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, _ *PlanCounterTp, _ *physicalOptimizeOp) (t task, cntPlan int64, err error) { +func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *PlanCounterTp, pop *physicalOptimizeOp) (t task, cntPlan int64, err error) { + if len(p.children) > 0 { + logutil.BgLogger().Warn("build physical for cte", zap.String("logical plan", ToString(p.children[0]))) + return p.baseLogicalPlan.findBestTask(prop, counter, pop) + } + logutil.BgLogger().Warn("build cte", zap.String("prop type", prop.TaskTp.String()), zap.Bool("can mpp", prop.CTECanMPP)) if !prop.IsSortItemEmpty() && !prop.CanAddEnforcer { return invalidTask, 1, nil } + logutil.BgLogger().Warn("build cte", zap.String("prop type", prop.TaskTp.String()), zap.Bool("can mpp", prop.CTECanMPP)) // The physical plan has been build when derive stats. pcte := PhysicalCTE{SeedPlan: p.cte.seedPartPhysicalPlan, RecurPlan: p.cte.recursivePartPhysicalPlan, CTE: p.cte, cteAsName: p.cteAsName, cteName: p.cteName}.Init(p.ctx, p.stats) pcte.SetSchema(p.schema) - t = &rootTask{pcte, false} + if prop.IsFlashProp() && prop.CTECanMPP { + t = &mppTask{ + p: pcte, + partTp: prop.MPPPartitionTp, + hashCols: prop.MPPPartitionCols, + tblColHists: p.stats.HistColl, + } + } else { + t = &rootTask{pcte, false} + } if prop.CanAddEnforcer { t = enforceProperty(prop, t, p.basePlan.ctx) } diff --git a/planner/core/hints.go b/planner/core/hints.go index baf9f91330d20..941f77d0a1386 100644 --- a/planner/core/hints.go +++ b/planner/core/hints.go @@ -174,7 +174,10 @@ func genHintsFromSingle(p PhysicalPlan, nodeType utilhint.NodeType, res []*ast.T } switch pp := p.(type) { case *PhysicalTableReader: - tbl := pp.TablePlans[0].(*PhysicalTableScan) + tbl, ok := pp.TablePlans[0].(*PhysicalTableScan) + if !ok { + return res + } if tbl.StoreType == kv.TiFlash { res = append(res, &ast.TableOptimizerHint{ QBName: qbName, diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 4a096b7b49204..148b65c4334ec 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -580,7 +580,7 @@ func (p LogicalCTE) Init(ctx sessionctx.Context, offset int) *LogicalCTE { // Init only assigns type and context. func (p PhysicalCTE) Init(ctx sessionctx.Context, stats *property.StatsInfo) *PhysicalCTE { - p.basePlan = newBasePlan(ctx, plancodec.TypeCTE, 0) + p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeCTE, &p, 0) p.stats = stats return &p } @@ -611,3 +611,15 @@ func (p FKCascade) Init(ctx sessionctx.Context) *FKCascade { p.stats = &property.StatsInfo{} return &p } + +func (p LogicalSequence) Init(ctx sessionctx.Context, offset int) *LogicalSequence { + p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeSequence, &p, offset) + return &p +} + +func (p PhysicalSequence) Init(ctx sessionctx.Context, stats *property.StatsInfo, blockOffset int, props ...*property.PhysicalProperty) *PhysicalSequence { + p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeSequence, &p, blockOffset) + p.stats = stats + p.childrenReqProps = props + return &p +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 2d73534fc2e1e..c674258f7d453 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -1649,7 +1649,7 @@ func (b *PlanBuilder) buildSetOpr(ctx context.Context, setOpr *ast.SetOprStmt) ( defer func() { b.outerCTEs = b.outerCTEs[:l] }() - err := b.buildWith(ctx, setOpr.With) + _, err := b.buildWith(ctx, setOpr.With) if err != nil { return nil, err } @@ -2639,7 +2639,7 @@ func (r *correlatedAggregateResolver) resolveSelect(sel *ast.SelectStmt) (err er defer func() { r.b.outerCTEs = r.b.outerCTEs[:l] }() - err := r.b.buildWith(r.ctx, sel.With) + _, err := r.b.buildWith(r.ctx, sel.With) if err != nil { return err } @@ -3932,12 +3932,13 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L } } + var currentLayerCTEs []*cteInfo if sel.With != nil { l := len(b.outerCTEs) defer func() { b.outerCTEs = b.outerCTEs[:l] }() - err = b.buildWith(ctx, sel.With) + currentLayerCTEs, err = b.buildWith(ctx, sel.With) if err != nil { return nil, err } @@ -4161,7 +4162,37 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L return proj, nil } - return p, nil + return b.tryToBuildSequence(currentLayerCTEs, p), nil +} + +func (b *PlanBuilder) tryToBuildSequence(ctes []*cteInfo, p LogicalPlan) LogicalPlan { + for i := len(ctes) - 1; i >= 0; i-- { + if !ctes[i].nonRecursive { + return p + } + if ctes[i].isInline { + ctes = append(ctes[:i], ctes[i+1:]...) + } + } + if len(ctes) == 0 { + return p + } + lctes := make([]LogicalPlan, 0, len(ctes)+1) + for _, cte := range ctes { + lcte := LogicalCTE{ + cte: cte.cteClass, + cteAsName: cte.def.Name, + cteName: cte.def.Name, + seedStat: cte.seedStat, + }.Init(b.ctx, b.getSelectOffset()) + lctes = append(lctes, lcte) + lcte.SetChildren(cte.seedLP) + } + seq := LogicalSequence{ + ctes: ctes, + }.Init(b.ctx, b.getSelectOffset()) + seq.SetChildren(append(lctes, p)...) + return seq } func (b *PlanBuilder) buildTableDual() *LogicalTableDual { @@ -5437,7 +5468,7 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( defer func() { b.outerCTEs = b.outerCTEs[:l] }() - err := b.buildWith(ctx, update.With) + _, err := b.buildWith(ctx, update.With) if err != nil { return nil, err } @@ -5844,7 +5875,7 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, ds *ast.DeleteStmt) (Plan defer func() { b.outerCTEs = b.outerCTEs[:l] }() - err := b.buildWith(ctx, ds.With) + _, err := b.buildWith(ctx, ds.With) if err != nil { return nil, err } @@ -7062,7 +7093,7 @@ func (b *PlanBuilder) buildRecursiveCTE(ctx context.Context, cte ast.ResultSetNo sw := x.With x.With = sw }() - err := b.buildWith(ctx, x.With) + _, err := b.buildWith(ctx, x.With) if err != nil { return err } @@ -7260,15 +7291,16 @@ func (b *PlanBuilder) genCTETableNameForError() string { return name } -func (b *PlanBuilder) buildWith(ctx context.Context, w *ast.WithClause) error { +func (b *PlanBuilder) buildWith(ctx context.Context, w *ast.WithClause) ([]*cteInfo, error) { // Check CTE name must be unique. nameMap := make(map[string]struct{}) for _, cte := range w.CTEs { if _, ok := nameMap[cte.Name.L]; ok { - return ErrNonUniqTable + return nil, ErrNonUniqTable } nameMap[cte.Name.L] = struct{}{} } + ctes := make([]*cteInfo, 0, len(w.CTEs)) for _, cte := range w.CTEs { b.outerCTEs = append(b.outerCTEs, &cteInfo{def: cte, nonRecursive: !w.IsRecursive, isBuilding: true, storageID: b.allocIDForCTEStorage, seedStat: &property.StatsInfo{}}) b.allocIDForCTEStorage++ @@ -7283,15 +7315,16 @@ func (b *PlanBuilder) buildWith(ctx context.Context, w *ast.WithClause) error { } _, err := b.buildCte(ctx, cte, w.IsRecursive) if err != nil { - return err + return nil, err } b.outerCTEs[len(b.outerCTEs)-1].optFlag = b.optFlag b.outerCTEs[len(b.outerCTEs)-1].isBuilding = false b.optFlag = saveFlag // each cte (select statement) will generate a handle map, pop it out here. b.handleHelper.popMap() + ctes = append(ctes, b.outerCTEs[len(b.outerCTEs)-1]) } - return nil + return ctes, nil } func (b *PlanBuilder) buildProjection4CTEUnion(_ context.Context, seed LogicalPlan, recur LogicalPlan) (LogicalPlan, error) { diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index f6bb7284e9afc..a7da6f9945181 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -2006,3 +2006,13 @@ func (p *LogicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn { } return corCols } + +type LogicalSequence struct { + baseLogicalPlan + + ctes []*cteInfo +} + +func (p *LogicalSequence) Schema() *expression.Schema { + return p.children[len(p.children)-1].Schema() +} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index a5db249fd2be1..f714291e45855 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -272,23 +272,16 @@ func checkStableResultMode(sctx sessionctx.Context) bool { // DoOptimize optimizes a logical plan to a physical plan. func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (PhysicalPlan, float64, error) { - // if there is something after flagPrunColumns, do flagPrunColumnsAgain - if flag&flagPrunColumns > 0 && flag-flagPrunColumns > flagPrunColumns { - flag |= flagPrunColumnsAgain - } - if checkStableResultMode(sctx) { - flag |= flagStabilizeResults - } - if sctx.GetSessionVars().StmtCtx.StraightJoinOrder { - // When we use the straight Join Order hint, we should disable the join reorder optimization. - flag &= ^flagJoinReOrder - } - flag |= flagCollectPredicateColumnsPoint - flag |= flagSyncWaitStatsLoadPoint + logic, err := logicalOptimize(ctx, flag, logic) if err != nil { return nil, 0, err } + + if !sctx.GetSessionVars().InRestrictedSQL { + logutil.BgLogger().Warn("optimizing", zap.String("plan", ToString(logic))) + } + if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, 0, errors.Trace(ErrCartesianProductUnsupported) } @@ -1039,6 +1032,19 @@ func LogicalOptimizeTest(ctx context.Context, flag uint64, logic LogicalPlan) (L } func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (LogicalPlan, error) { + // if there is something after flagPrunColumns, do flagPrunColumnsAgain + if flag&flagPrunColumns > 0 && flag-flagPrunColumns > flagPrunColumns { + flag |= flagPrunColumnsAgain + } + if checkStableResultMode(logic.SCtx()) { + flag |= flagStabilizeResults + } + if logic.SCtx().GetSessionVars().StmtCtx.StraightJoinOrder { + // When we use the straight Join Order hint, we should disable the join reorder optimization. + flag &= ^flagJoinReOrder + } + flag |= flagCollectPredicateColumnsPoint + flag |= flagSyncWaitStatsLoadPoint opt := defaultLogicalOptimizeOption() vars := logic.SCtx().GetSessionVars() if vars.StmtCtx.EnableOptimizeTrace { diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 02d01aad5db4c..22f85588ed9c9 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -2453,6 +2453,33 @@ func (p *CTEDefinition) MemoryUsage() (sum int64) { return } +type PhysicalCTEStorage PhysicalCTE + +// ExplainInfo overrides the ExplainInfo +func (p *PhysicalCTEStorage) ExplainInfo() string { + return "Non-Recursive CTE Storage" +} + +// ExplainID overrides the ExplainID. +func (p *PhysicalCTEStorage) ExplainID() fmt.Stringer { + return stringutil.MemoizeStr(func() string { + return "CTE_" + strconv.Itoa(p.CTE.IDForStorage) + }) +} + +// MemoryUsage return the memory usage of CTEDefinition +func (p *PhysicalCTEStorage) MemoryUsage() (sum int64) { + if p == nil { + return + } + + sum = p.physicalSchemaProducer.MemoryUsage() + p.cteAsName.MemoryUsage() + if p.CTE != nil { + sum += p.CTE.MemoryUsage() + } + return +} + func appendChildCandidate(origin PhysicalPlan, pp PhysicalPlan, op *physicalOptimizeOp) { candidate := &tracing.CandidatePlanTrace{ PlanTrace: &tracing.PlanTrace{ @@ -2466,3 +2493,37 @@ func appendChildCandidate(origin PhysicalPlan, pp PhysicalPlan, op *physicalOpti pp.appendChildCandidate(op) op.tracer.Candidates[origin.ID()].AppendChildrenID(pp.ID()) } + +type PhysicalSequence struct { + physicalSchemaProducer + + ctes []*PhysicalCTE +} + +func (p *PhysicalSequence) MemoryUsage() (sum int64) { + if p == nil { + return + } + + sum = p.physicalSchemaProducer.MemoryUsage() + + for _, cte := range p.ctes { + sum += cte.MemoryUsage() + } + return +} + +func (p *PhysicalSequence) ExplainID() fmt.Stringer { + return stringutil.MemoizeStr(func() string { + if p.ctx != nil && p.ctx.GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { + return p.TP() + } + return p.TP() + "_" + strconv.Itoa(p.id) + }) +} + +func (p *PhysicalSequence) ExplainInfo() string { + var res string + res = "Sequence Node" + return res +} diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 5af0bb004d6c9..828dead56c877 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1315,6 +1315,11 @@ func getPossibleAccessPaths(ctx sessionctx.Context, tableHints *tableHintInfo, i publicPaths = append(publicPaths, tablePath) if tblInfo.TiFlashReplica == nil { + + if !ctx.GetSessionVars().InRestrictedSQL { + publicPaths = append(publicPaths, genTiFlashPath(tblInfo)) + } + ctx.GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because there aren't tiflash replicas of table `" + tblInfo.Name.O + "`.") } else if !tblInfo.TiFlashReplica.Available { ctx.GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because tiflash replicas of table `" + tblInfo.Name.O + "` not ready.") diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 34a5259abbd32..973714c7c3c54 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -666,3 +666,13 @@ func preferKeyColumnFromTable(dataSource *DataSource, originColumns []*expressio } return resultColumn, resultColumnInfo } + +// PruneColumns implements the interface of LogicalPlan. +// LogicalCTE just do a empty function call. It's logical optimize is indivisual phase. +func (p *LogicalCTE) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { + return nil +} + +func (p *LogicalSequence) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { + return p.children[len(p.children)-1].PruneColumns(parentUsedCols, opt) +} diff --git a/planner/core/rule_decorrelate.go b/planner/core/rule_decorrelate.go index 5d5fcef35c3e2..686fc18edfd11 100644 --- a/planner/core/rule_decorrelate.go +++ b/planner/core/rule_decorrelate.go @@ -371,6 +371,10 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo } } NoOptimize: + // CTE's logical optimization is independent. + if _, ok := p.(*LogicalCTE); ok { + return p, nil + } newChildren := make([]LogicalPlan, 0, len(p.Children())) for _, child := range p.Children() { np, err := s.optimize(ctx, child, opt) diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 83936e7cd9e94..7a9d49bff2496 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -168,6 +168,10 @@ func (pe *projectionEliminator) optimize(_ context.Context, lp LogicalPlan, opt // eliminate eliminates the redundant projection in a logical plan. func (pe *projectionEliminator) eliminate(p LogicalPlan, replace map[string]*expression.Column, canEliminate bool, opt *logicalOptimizeOp) LogicalPlan { + // LogicalCTE's logical optimization is independent. + if _, ok := p.(*LogicalCTE); ok { + return p + } proj, isProj := p.(*LogicalProjection) childFlag := canEliminate if _, isUnion := p.(*LogicalUnionAll); isUnion { diff --git a/planner/core/rule_generate_column_substitute.go b/planner/core/rule_generate_column_substitute.go index c796f99af62c5..610f9fbf47c04 100644 --- a/planner/core/rule_generate_column_substitute.go +++ b/planner/core/rule_generate_column_substitute.go @@ -50,6 +50,9 @@ func (gc *gcSubstituter) optimize(ctx context.Context, lp LogicalPlan, _ *logica // For the sake of simplicity, we don't collect the stored generate column because we can't get their expressions directly. // TODO: support stored generate column. func collectGenerateColumn(lp LogicalPlan, exprToColumn ExprColumnMap) { + if _, ok := lp.(*LogicalCTE); ok { + return + } for _, child := range lp.Children() { collectGenerateColumn(child, exprToColumn) } diff --git a/planner/core/rule_join_elimination.go b/planner/core/rule_join_elimination.go index 330b0ba8b98bb..8e7731b338c74 100644 --- a/planner/core/rule_join_elimination.go +++ b/planner/core/rule_join_elimination.go @@ -184,6 +184,10 @@ func GetDupAgnosticAggCols( } func (o *outerJoinEliminator) doOptimize(p LogicalPlan, aggCols []*expression.Column, parentCols []*expression.Column, opt *logicalOptimizeOp) (LogicalPlan, error) { + // CTE's logical optimization is independent. + if _, ok := p.(*LogicalCTE); ok { + return p, nil + } var err error var isEliminated bool for join, isJoin := p.(*LogicalJoin); isJoin; join, isJoin = p.(*LogicalJoin) { diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 2c3b603db12c9..bae5790741508 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -176,6 +176,10 @@ func (s *joinReOrderSolver) optimize(_ context.Context, p LogicalPlan, opt *logi // optimizeRecursive recursively collects join groups and applies join reorder algorithm for each group. func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalPlan, tracer *joinReorderTrace) (LogicalPlan, error) { + if _, ok := p.(*LogicalCTE); ok { + return p, nil + } + var err error curJoinGroup, eqEdges, otherConds, joinTypes, hintInfo, hasOuterJoin := extractJoinGroup(p) diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index d9bb1eae19f9b..de5ab2e35bf5e 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -205,6 +205,10 @@ func (a *maxMinEliminator) eliminateSingleMaxMin(agg *LogicalAggregation, opt *l // eliminateMaxMin tries to convert max/min to Limit+Sort operators. func (a *maxMinEliminator) eliminateMaxMin(p LogicalPlan, opt *logicalOptimizeOp) LogicalPlan { + // CTE's logical optimization is indenpent. + if _, ok := p.(*LogicalCTE); ok { + return p + } newChildren := make([]LogicalPlan, 0, len(p.Children())) for _, child := range p.Children() { newChildren = append(newChildren, a.eliminateMaxMin(child, opt)) diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 81a43935b056f..495ef59127602 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -94,6 +94,8 @@ func (s *partitionProcessor) rewriteDataSource(lp LogicalPlan, opt *logicalOptim // Only one partition, no union all. p.SetChildren(ds) return p, nil + case *LogicalCTE: + return lp, nil default: children := lp.Children() for i, child := range children { diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 94f92f780202a..b93075668b832 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -1005,3 +1005,10 @@ func (p *LogicalCTE) PredicatePushDown(predicates []expression.Expression, _ *lo p.cte.pushDownPredicates = append(p.cte.pushDownPredicates, expression.ComposeCNFCondition(p.ctx, newPred...)) return predicates, p.self } + +func (p *LogicalSequence) PredicatePushDown(predicates []expression.Expression, op *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { + lastIdx := len(p.children) - 1 + remained, newLastChild := p.children[lastIdx].PredicatePushDown(predicates, op) + p.SetChild(lastIdx, newLastChild) + return remained, p +} diff --git a/planner/core/rule_semi_join_rewrite.go b/planner/core/rule_semi_join_rewrite.go index e38155d8ef455..25e4f624ddfe5 100644 --- a/planner/core/rule_semi_join_rewrite.go +++ b/planner/core/rule_semi_join_rewrite.go @@ -34,6 +34,9 @@ func (*semiJoinRewriter) name() string { } func (smj *semiJoinRewriter) recursivePlan(p LogicalPlan) (LogicalPlan, error) { + if _, ok := p.(*LogicalCTE); ok { + return p, nil + } newChildren := make([]LogicalPlan, 0, len(p.Children())) for _, child := range p.Children() { newChild, err := smj.recursivePlan(child) diff --git a/planner/core/rule_topn_push_down.go b/planner/core/rule_topn_push_down.go index 5dacac7579caa..f6e9fc40ab7ab 100644 --- a/planner/core/rule_topn_push_down.go +++ b/planner/core/rule_topn_push_down.go @@ -43,6 +43,13 @@ func (s *baseLogicalPlan) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp return p } +func (p *LogicalCTE) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) LogicalPlan { + if topN != nil { + return topN.setChild(p, opt) + } + return p +} + // setChild set p as topn's child. func (lt *LogicalTopN) setChild(p LogicalPlan, opt *logicalOptimizeOp) LogicalPlan { // Remove this TopN if its child is a TableDual. diff --git a/planner/core/stats.go b/planner/core/stats.go index 96065fccf12b5..7b94ac4ec5bc8 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -980,11 +980,16 @@ func (p *LogicalWindow) ExtractColGroups(colGroups [][]*expression.Column) [][]* } // DeriveStats implement LogicalPlan DeriveStats interface. -func (p *LogicalCTE) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { +func (p *LogicalCTE) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { if p.stats != nil { return p.stats, nil } + if len(p.children) > 0 { + p.stats = childStats[0] + return p.stats, nil + } + var err error if p.cte.seedPartPhysicalPlan == nil { // Build push-downed predicates. @@ -1037,3 +1042,31 @@ func (p *LogicalCTETable) DeriveStats(_ []*property.StatsInfo, _ *expression.Sch p.stats = p.seedStat return p.stats, nil } + +func (p *LogicalSequence) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { + p.stats = childStats[len(childStats)-1] + return p.stats, nil +} + +func (p *LogicalCTE) recursiveDeriveStats(colGroups [][]*expression.Column) (*property.StatsInfo, error) { + if len(p.children) == 0 { + return p.DeriveStats(nil, p.self.Schema(), nil, colGroups) + } + var err error + p.children[0], err = logicalOptimize(context.TODO(), p.cte.optFlag, p.children[0]) + if err != nil { + return nil, err + } + childStats := make([]*property.StatsInfo, len(p.children)) + childSchema := make([]*expression.Schema, len(p.children)) + cumColGroups := p.self.ExtractColGroups(colGroups) + for i, child := range p.children { + childProfile, err := child.recursiveDeriveStats(cumColGroups) + if err != nil { + return nil, err + } + childStats[i] = childProfile + childSchema[i] = child.Schema() + } + return p.self.DeriveStats(childStats, p.self.Schema(), childSchema, colGroups) +} diff --git a/planner/core/stringer.go b/planner/core/stringer.go index 3679b92ce777c..6d185146d414f 100644 --- a/planner/core/stringer.go +++ b/planner/core/stringer.go @@ -201,6 +201,14 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { } str = name + "{" + strings.Join(children, "->") + "}" idxs = idxs[:last] + case *LogicalSequence: + last := len(idxs) - 1 + idx := idxs[last] + children := strs[idx:] + strs = strs[:idx] + name := "Sequence" + str = name + "{" + strings.Join(children, ",") + "}" + idxs = idxs[:last] case *DataSource: if x.isPartition { str = fmt.Sprintf("Partition(%d)", x.physicalTableID) diff --git a/planner/core/task.go b/planner/core/task.go index 5d7ca6e5fd424..bd55f843c4616 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -2058,6 +2058,51 @@ func (p *PhysicalWindow) attach2Task(tasks ...task) task { return attachPlan2Task(p.self, t) } +func (p *PhysicalCTEStorage) attach2Task(tasks ...task) task { + t := tasks[0].copy() + if mpp, ok := t.(*mppTask); ok { + p.SetChildren(t.plan()) + return &mppTask{ + p: p, + partTp: mpp.partTp, + hashCols: mpp.hashCols, + tblColHists: mpp.tblColHists, + } + } + t.convertToRootTask(p.ctx) + p.SetChildren(t.plan()) + return &rootTask{ + p: p, + } +} + +func (p *PhysicalSequence) attach2Task(tasks ...task) task { + for _, t := range tasks { + _, isMpp := t.(*mppTask) + if !isMpp { + return tasks[len(tasks)-1] + } + } + + lastTask := tasks[len(tasks)-1].(*mppTask) + + children := make([]PhysicalPlan, 0, len(tasks)) + for _, t := range tasks { + children = append(children, t.plan()) + } + + p.SetChildren(children...) + + mppTask := &mppTask{ + p: p, + partTp: lastTask.partTp, + hashCols: lastTask.hashCols, + tblColHists: lastTask.tblColHists, + } + return mppTask + +} + // mppTask can not : // 1. keep order // 2. support double read diff --git a/planner/property/physical_property.go b/planner/property/physical_property.go index 60994d05b57ef..a22ccbe7466c6 100644 --- a/planner/property/physical_property.go +++ b/planner/property/physical_property.go @@ -202,6 +202,8 @@ type PhysicalProperty struct { // RejectSort means rejecting the sort property from its children, but it only works for MPP tasks. // Non-MPP tasks do not care about it. RejectSort bool + + CTECanMPP bool } // NewPhysicalProperty builds property from columns. @@ -328,6 +330,11 @@ func (p *PhysicalProperty) HashCode() []byte { for _, col := range p.MPPPartitionCols { p.hashcode = append(p.hashcode, col.hashCode(nil)...) } + if p.CTECanMPP { + p.hashcode = append(p.hashcode, codec.EncodeInt(p.hashcode, 1)...) + } else { + p.hashcode = append(p.hashcode, codec.EncodeInt(p.hashcode, 0)...) + } } return p.hashcode } @@ -348,6 +355,7 @@ func (p *PhysicalProperty) CloneEssentialFields() *PhysicalProperty { MPPPartitionTp: p.MPPPartitionTp, MPPPartitionCols: p.MPPPartitionCols, RejectSort: p.RejectSort, + CTECanMPP: p.CTECanMPP, } return prop } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 0b6e7ef0cb0d0..a8ff6ae274c9e 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -690,6 +690,9 @@ type SessionVars struct { // PlanColumnID is the unique id for column when building plan. PlanColumnID int64 + // CurrentCTECanMPP indicates we can make the current cte operator executed on MPP engine. + CurrentCTECanMPP bool + // MapHashCode2UniqueID4ExtendedCol map the expr's hash code to specified unique ID. MapHashCode2UniqueID4ExtendedCol map[string]int diff --git a/util/plancodec/id.go b/util/plancodec/id.go index 0366b7dc3f5a6..7f8bf0b56600e 100644 --- a/util/plancodec/id.go +++ b/util/plancodec/id.go @@ -131,6 +131,8 @@ const ( TypeForeignKeyCheck = "Foreign_Key_Check" // TypeForeignKeyCascade is the type of FKCascade TypeForeignKeyCascade = "Foreign_Key_Cascade" + // TypeSequence + TypeSequence = "Sequence" ) // plan id. From 95375d799e340df7179a0fc8fa4b0a24d5b75946 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 22 Feb 2023 17:53:43 +0000 Subject: [PATCH 02/24] fix the mpp prop --- planner/core/find_best_task.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 15b97517fde20..4cba203c67fc7 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -2643,6 +2643,9 @@ func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *Plan pcte := PhysicalCTE{SeedPlan: p.cte.seedPartPhysicalPlan, RecurPlan: p.cte.recursivePartPhysicalPlan, CTE: p.cte, cteAsName: p.cteAsName, cteName: p.cteName}.Init(p.ctx, p.stats) pcte.SetSchema(p.schema) if prop.IsFlashProp() && prop.CTECanMPP { + if prop.MPPPartitionTp != property.AnyType { + return invalidTask, 1, nil + } t = &mppTask{ p: pcte, partTp: prop.MPPPartitionTp, From 5584c9a4207e44e2e6c3659295aaa961ec12055a Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 1 Mar 2023 08:24:23 +0000 Subject: [PATCH 03/24] planner: support sending cte mpp task --- executor/mpp_gather.go | 11 ++++ go.mod | 1 + go.sum | 4 +- planner/core/common_plans.go | 5 +- planner/core/find_best_task.go | 3 +- planner/core/fragment.go | 111 ++++++++++++++++++++++++++++++--- planner/core/physical_plans.go | 54 +++++++++++++--- planner/core/plan_to_pb.go | 30 ++++++--- planner/core/stringer.go | 56 +++++++++++++++++ 9 files changed, 246 insertions(+), 29 deletions(-) diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index b2ced590dc338..6c67dfb8ee1db 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -88,6 +88,16 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { } else { dagReq.EncodeType = tipb.EncodeType_TypeChunk } + // for _, mppTask := range pf.ExchangeSender.Tasks { + // logutil.BgLogger().Info("Dispatch mpp task", zap.Uint64("timestamp", mppTask.StartTs), + // zap.Int64("ID", mppTask.ID), zap.Uint64("QueryTs", mppTask.MppQueryID.QueryTs), zap.Uint64("LocalQueryId", mppTask.MppQueryID.LocalQueryID), + // zap.Uint64("ServerID", mppTask.MppQueryID.ServerID), zap.String("address", mppTask.Meta.GetAddress()), + // zap.String("plan", plannercore.ToStringNewForMPP(pf.ExchangeSender)), + // zap.Int64("mpp-version", mppTask.MppVersion.ToInt64()), + // zap.String("exchange-compression-mode", pf.ExchangeSender.CompressionMode.Name()), + // ) + // } + // return nil for _, mppTask := range pf.ExchangeSender.Tasks { if mppTask.PartitionTableIDs != nil { err = updateExecutorTableID(context.Background(), dagReq.RootExecutor, true, mppTask.PartitionTableIDs) @@ -151,6 +161,7 @@ func (e *MPPGather) Open(ctx context.Context) (err error) { return errors.Trace(err) } } + // return errors.Errorf("break for debugging") failpoint.Inject("checkTotalMPPTasks", func(val failpoint.Value) { if val.(int) != len(e.mppReqs) { failpoint.Return(errors.Errorf("The number of tasks is not right, expect %d tasks but actually there are %d tasks", val.(int), len(e.mppReqs))) diff --git a/go.mod b/go.mod index 0338f7acb4cb1..65034c8b88988 100644 --- a/go.mod +++ b/go.mod @@ -277,5 +277,6 @@ replace ( // fix potential security issue(CVE-2020-26160) introduced by indirect dependency. github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible github.com/pingcap/tidb/parser => ./parser + github.com/pingcap/tipb => github.com/pingcap/tipb v0.0.0-20230301075649-6f61c6d79ef3 go.opencensus.io => go.opencensus.io v0.23.1-0.20220331163232-052120675fac ) diff --git a/go.sum b/go.sum index 751ec3487c08a..37e0e88a1f28b 100644 --- a/go.sum +++ b/go.sum @@ -781,8 +781,8 @@ github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c h1:crhkw6DD+07Bg1wYh github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops= -github.com/pingcap/tipb v0.0.0-20230119054146-c6b7a5a1623b h1:j5sw2YZY7QfgIFZEoUcn1P5cYflms1PCVVS96i+IQiI= -github.com/pingcap/tipb v0.0.0-20230119054146-c6b7a5a1623b/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20230301075649-6f61c6d79ef3 h1:pmrwFxfD9t3wvXHJ5z9+UPZX5pc/jtEVemDFV8Sy3/E= +github.com/pingcap/tipb v0.0.0-20230301075649-6f61c6d79ef3/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index e84c1c50d9033..e9dbd1767cda8 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -1321,7 +1321,10 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p Plan) (bo indexScan := v.IndexPlans[0].(*PhysicalIndexScan) return indexScan.IsPointGetByUniqueKey(ctx), nil case *PhysicalTableReader: - tableScan := v.TablePlans[0].(*PhysicalTableScan) + tableScan, ok := v.TablePlans[0].(*PhysicalTableScan) + if !ok { + return false, nil + } isPointRange := len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPointNonNullable(ctx) if !isPointRange { return false, nil diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 37df3c259a078..092918a506a14 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -2665,14 +2665,13 @@ func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *Plan logutil.BgLogger().Warn("build physical for cte", zap.String("logical plan", ToString(p.children[0]))) return p.baseLogicalPlan.findBestTask(prop, counter, pop) } - logutil.BgLogger().Warn("build cte", zap.String("prop type", prop.TaskTp.String()), zap.Bool("can mpp", prop.CTECanMPP)) if !prop.IsSortItemEmpty() && !prop.CanAddEnforcer { return invalidTask, 1, nil } - logutil.BgLogger().Warn("build cte", zap.String("prop type", prop.TaskTp.String()), zap.Bool("can mpp", prop.CTECanMPP)) // The physical plan has been build when derive stats. pcte := PhysicalCTE{SeedPlan: p.cte.seedPartPhysicalPlan, RecurPlan: p.cte.recursivePartPhysicalPlan, CTE: p.cte, cteAsName: p.cteAsName, cteName: p.cteName}.Init(p.ctx, p.stats) pcte.SetSchema(p.schema) + logutil.BgLogger().Warn("build cte", zap.String("prop type", prop.TaskTp.String()), zap.Bool("can mpp", prop.CTECanMPP), zap.String("cte name", pcte.ExplainID().String())) if prop.IsFlashProp() && prop.CTECanMPP { if prop.MPPPartitionTp != property.AnyType { return invalidTask, 1, nil diff --git a/planner/core/fragment.go b/planner/core/fragment.go index a712f93b418ae..88777f776cac9 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -44,6 +45,7 @@ type Fragment struct { // following field are filled during getPlanFragment. TableScan *PhysicalTableScan // result physical table scan ExchangeReceivers []*PhysicalExchangeReceiver // data receivers + CTEReaders []*PhysicalCTE // The receivers for CTE storage/producer. // following fields are filled after scheduling. ExchangeSender *PhysicalExchangeSender // data exporter @@ -53,6 +55,14 @@ type Fragment struct { singleton bool // indicates if this is a task running on a single node. } +type CTEGroupInFragment struct { + CTEStorage *PhysicalCTEStorage + CTEReader []*PhysicalCTE + + StroageTasks []*kv.MPPTask + StroageFragments []*Fragment +} + const emptyFragmentSize = int64(unsafe.Sizeof(Fragment{})) // MemoryUsage return the memory usage of Fragment @@ -87,6 +97,8 @@ type mppTaskGenerator struct { is infoschema.InfoSchema frags []*Fragment cache map[int]tasksAndFrags + + CTEGroups map[int]*CTEGroupInFragment } // GenerateRootMPPTasks generate all mpp tasks and return root ones. @@ -116,7 +128,6 @@ func AllocMPPQueryID() uint64 { func (e *mppTaskGenerator) generateMPPTasks(s *PhysicalExchangeSender) ([]*Fragment, error) { mppVersion := e.ctx.GetSessionVars().ChooseMppVersion() - logutil.BgLogger().Info("Mpp will generate tasks", zap.String("plan", ToString(s)), zap.Int64("mpp-version", mppVersion.ToInt64())) tidbTask := &kv.MPPTask{ StartTs: e.startTS, MppQueryID: e.mppQueryID, @@ -182,6 +193,8 @@ func (f *Fragment) init(p PhysicalPlan) error { f.ExchangeReceivers = append(f.ExchangeReceivers, x) case *PhysicalUnionAll: return errors.New("unexpected union all detected") + case *PhysicalCTE: + f.CTEReaders = append(f.CTEReaders, x) default: for _, ch := range p.Children() { if err := f.init(ch); err != nil { @@ -200,10 +213,10 @@ func (f *Fragment) init(p PhysicalPlan) error { // after untwist, there will be two plans in `forest` slice: // - ExchangeSender -> Projection (c1) -> TableScan(t) // - ExchangeSender -> Projection (c2) -> TableScan(s) -func untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, forest *[]*PhysicalExchangeSender) error { +func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, forest *[]*PhysicalExchangeSender) error { cur := stack[len(stack)-1] switch x := cur.(type) { - case *PhysicalTableScan, *PhysicalExchangeReceiver: // This should be the leave node. + case *PhysicalTableScan, *PhysicalExchangeReceiver, *PhysicalCTE: // This should be the leave node. p, err := stack[0].Clone() if err != nil { return errors.Trace(err) @@ -213,6 +226,9 @@ func untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, forest *[]*PhysicalExcha if _, ok := stack[i].(*PhysicalUnionAll); ok { continue } + if _, ok := stack[i].(*PhysicalSequence); ok { + continue + } ch, err := stack[i].Clone() if err != nil { return errors.Trace(err) @@ -224,36 +240,57 @@ func untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, forest *[]*PhysicalExcha } p = ch } + if cte, ok := p.(*PhysicalCTE); ok { + e.CTEGroups[cte.CTE.IDForStorage].CTEReader = append(e.CTEGroups[cte.CTE.IDForStorage].CTEReader, cte) + } case *PhysicalHashJoin: stack = append(stack, x.children[1-x.InnerChildIdx]) - err := untwistPlanAndRemoveUnionAll(stack, forest) + err := e.untwistPlanAndRemoveUnionAll(stack, forest) stack = stack[:len(stack)-1] return errors.Trace(err) case *PhysicalUnionAll: for _, ch := range x.children { stack = append(stack, ch) - err := untwistPlanAndRemoveUnionAll(stack, forest) + err := e.untwistPlanAndRemoveUnionAll(stack, forest) stack = stack[:len(stack)-1] if err != nil { return errors.Trace(err) } } + case *PhysicalSequence: + lastChildIdx := len(x.children) - 1 + for i := 0; i < lastChildIdx; i++ { + if e.CTEGroups == nil { + e.CTEGroups = make(map[int]*CTEGroupInFragment) + } + cteStorage := x.children[i].(*PhysicalCTEStorage) + e.CTEGroups[cteStorage.CTE.IDForStorage] = &CTEGroupInFragment{ + CTEStorage: cteStorage, + CTEReader: make([]*PhysicalCTE, 0, 3), + } + } + stack = append(stack, x.children[lastChildIdx]) + err := e.untwistPlanAndRemoveUnionAll(stack, forest) + stack = stack[:len(stack)-1] + if err != nil { + return err + } default: if len(cur.Children()) != 1 { return errors.Trace(errors.New("unexpected plan " + cur.ExplainID().String())) } ch := cur.Children()[0] stack = append(stack, ch) - err := untwistPlanAndRemoveUnionAll(stack, forest) + err := e.untwistPlanAndRemoveUnionAll(stack, forest) stack = stack[:len(stack)-1] return errors.Trace(err) } return nil } -func buildFragments(s *PhysicalExchangeSender) ([]*Fragment, error) { +func (e *mppTaskGenerator) buildFragments(s *PhysicalExchangeSender) ([]*Fragment, error) { forest := make([]*PhysicalExchangeSender, 0, 1) - err := untwistPlanAndRemoveUnionAll([]PhysicalPlan{s}, &forest) + err := e.untwistPlanAndRemoveUnionAll([]PhysicalPlan{s}, &forest) if err != nil { return nil, errors.Trace(err) } @@ -273,7 +310,9 @@ func (e *mppTaskGenerator) generateMPPTasksForExchangeSender(s *PhysicalExchange if cached, ok := e.cache[s.ID()]; ok { return cached.tasks, cached.frags, nil } - frags, err := buildFragments(s) + mppVersion := e.ctx.GetSessionVars().ChooseMppVersion() + logutil.BgLogger().Info("Mpp will generate tasks", zap.String("plan", ToStringNewForMPP(s)), zap.Int64("mpp-version", mppVersion.ToInt64())) + frags, err := e.buildFragments(s) if err != nil { return nil, nil, errors.Trace(err) } @@ -292,6 +331,9 @@ func (e *mppTaskGenerator) generateMPPTasksForExchangeSender(s *PhysicalExchange } func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv.MPPTask, err error) { + for _, cteReader := range f.CTEReaders { + e.generateTasksForCTEReader(cteReader) + } for _, r := range f.ExchangeReceivers { // chain call: to get lower fragments and tasks r.Tasks, r.frags, err = e.generateMPPTasksForExchangeSender(r.GetExchangeSender()) @@ -311,6 +353,9 @@ func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv for _, r := range f.ExchangeReceivers { childrenTasks = append(childrenTasks, r.Tasks...) } + for _, cteR := range f.CTEReaders { + childrenTasks = append(childrenTasks, cteR.children[0].(*PhysicalExchangeReceiver).Tasks...) + } if f.singleton && len(childrenTasks) > 0 { childrenTasks = childrenTasks[0:1] } @@ -324,10 +369,58 @@ func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv frag.ExchangeSender.TargetTasks = append(frag.ExchangeSender.TargetTasks, tasks...) } } + for _, cteR := range f.CTEReaders { + e.addReaderTasksForCTEStorage(cteR.CTE.IDForStorage, tasks...) + } f.ExchangeSender.Tasks = tasks + f.flipCTEReader(f.ExchangeSender) return tasks, nil } +func (f *Fragment) flipCTEReader(currentPlan PhysicalPlan) { + newChildren := make([]PhysicalPlan, len(currentPlan.Children())) + for i := 0; i < len(currentPlan.Children()); i++ { + child := currentPlan.Children()[i] + newChildren[i] = child + if cteR, ok := child.(*PhysicalCTE); ok { + receiver := cteR.Children()[0] + receiver.SetChildren(cteR) + cteR.SetChildren(nil) + newChildren[i] = receiver + } else if _, ok := child.(*PhysicalExchangeReceiver); !ok { + // The receiver is the leaf of the fragment though it has child, we need break it. + f.flipCTEReader(child) + } + } + currentPlan.SetChildren(newChildren...) +} + +func (e *mppTaskGenerator) generateTasksForCTEReader(cteReader *PhysicalCTE) (err error) { + group := e.CTEGroups[cteReader.CTE.IDForStorage] + if group.StroageFragments == nil { + exchangeSender := PhysicalExchangeSender{ + ExchangeType: tipb.ExchangeType(property.BroadcastType), + }.Init(group.CTEStorage.SCtx(), group.CTEStorage.children[0].statsInfo()) + exchangeSender.SetChildren(group.CTEStorage.children[0]) + group.StroageTasks, group.StroageFragments, err = e.generateMPPTasksForExchangeSender(exchangeSender) + if err != nil { + return err + } + } + receiver := PhysicalExchangeReceiver{}.Init(cteReader.SCtx(), group.CTEStorage.statsInfo()) + receiver.Tasks = group.StroageTasks + receiver.frags = group.StroageFragments + cteReader.SetChildren(receiver) + return nil +} + +func (e *mppTaskGenerator) addReaderTasksForCTEStorage(storageID int, tasks ...*kv.MPPTask) { + group := e.CTEGroups[storageID] + for _, frag := range group.StroageFragments { + frag.ExchangeSender.TargetCTEReaderTasks = append(frag.ExchangeSender.TargetCTEReaderTasks, tasks) + } +} + func partitionPruning(ctx sessionctx.Context, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []model.CIStr, columns []*expression.Column, columnNames types.NameSlice) ([]table.PhysicalTable, error) { idxArr, err := PartitionPruning(ctx, tbl, conds, partitionNames, columns, columnNames) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 11f9565e529cc..8b0afc946e72f 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1567,9 +1567,10 @@ func (p *PhysicalExpand) MemoryUsage() (sum int64) { type PhysicalExchangeSender struct { basePhysicalPlan - TargetTasks []*kv.MPPTask - ExchangeType tipb.ExchangeType - HashCols []*property.MPPPartitionColumn + TargetTasks []*kv.MPPTask + TargetCTEReaderTasks [][]*kv.MPPTask + ExchangeType tipb.ExchangeType + HashCols []*property.MPPPartitionColumn // Tasks is the mpp task for current PhysicalExchangeSender. Tasks []*kv.MPPTask CompressionMode kv.ExchangeCompressionMode @@ -2457,6 +2458,30 @@ func (p *PhysicalCTE) ExplainID() fmt.Stringer { }) } +func (p *PhysicalCTE) Clone() (PhysicalPlan, error) { + cloned := new(PhysicalCTE) + base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) + if err != nil { + return nil, err + } + cloned.physicalSchemaProducer = *base + if p.SeedPlan != nil { + cloned.SeedPlan, err = p.SeedPlan.Clone() + if err != nil { + return nil, err + } + } + if p.RecurPlan != nil { + cloned.RecurPlan, err = p.RecurPlan.Clone() + if err != nil { + return nil, err + } + } + cloned.cteAsName, cloned.cteName = p.cteAsName, p.cteName + cloned.CTE = p.CTE + return cloned, nil +} + // MemoryUsage return the memory usage of PhysicalCTE func (p *PhysicalCTE) MemoryUsage() (sum int64) { if p == nil { @@ -2560,6 +2585,14 @@ func (p *PhysicalCTEStorage) MemoryUsage() (sum int64) { return } +func (p *PhysicalCTEStorage) Clone() (PhysicalPlan, error) { + cloned, err := (*PhysicalCTE)(p).Clone() + if err != nil { + return nil, err + } + return (*PhysicalCTEStorage)(cloned.(*PhysicalCTE)), nil +} + func appendChildCandidate(origin PhysicalPlan, pp PhysicalPlan, op *physicalOptimizeOp) { candidate := &tracing.CandidatePlanTrace{ PlanTrace: &tracing.PlanTrace{ @@ -2576,8 +2609,6 @@ func appendChildCandidate(origin PhysicalPlan, pp PhysicalPlan, op *physicalOpti type PhysicalSequence struct { physicalSchemaProducer - - ctes []*PhysicalCTE } func (p *PhysicalSequence) MemoryUsage() (sum int64) { @@ -2587,9 +2618,6 @@ func (p *PhysicalSequence) MemoryUsage() (sum int64) { sum = p.physicalSchemaProducer.MemoryUsage() - for _, cte := range p.ctes { - sum += cte.MemoryUsage() - } return } @@ -2607,3 +2635,13 @@ func (p *PhysicalSequence) ExplainInfo() string { res = "Sequence Node" return res } + +func (p *PhysicalSequence) Clone() (PhysicalPlan, error) { + cloned := new(PhysicalSequence) + base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) + if err != nil { + return nil, err + } + cloned.physicalSchemaProducer = *base + return cloned, nil +} diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 8b385d3d86a43..f3ecf7734ddb1 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -302,6 +302,21 @@ func (e *PhysicalExchangeSender) ToPB(ctx sessionctx.Context, storeType kv.Store encodedTask = append(encodedTask, encodedStr) } + encodedUpstreamCTETask := make([]*tipb.EncodedBytesSlice, 0, len(e.TargetCTEReaderTasks)) + for _, cteRTasks := range e.TargetCTEReaderTasks { + encodedTasksForOneCTEReader := &tipb.EncodedBytesSlice{ + EncodedTasks: make([][]byte, 0, len(cteRTasks)), + } + for _, task := range cteRTasks { + encodedStr, err := task.ToPB().Marshal() + if err != nil { + return nil, err + } + encodedTasksForOneCTEReader.EncodedTasks = append(encodedTasksForOneCTEReader.EncodedTasks, encodedStr) + } + encodedUpstreamCTETask = append(encodedUpstreamCTETask, encodedTasksForOneCTEReader) + } + hashCols := make([]expression.Expression, 0, len(e.HashCols)) hashColTypes := make([]*tipb.FieldType, 0, len(e.HashCols)) for _, col := range e.HashCols { @@ -326,13 +341,14 @@ func (e *PhysicalExchangeSender) ToPB(ctx sessionctx.Context, storeType kv.Store return nil, errors.Trace(err) } ecExec := &tipb.ExchangeSender{ - Tp: e.ExchangeType, - EncodedTaskMeta: encodedTask, - PartitionKeys: hashColPb, - Child: child, - Types: hashColTypes, - AllFieldTypes: allFieldTypes, - Compression: e.CompressionMode.ToTipbCompressionMode(), + Tp: e.ExchangeType, + EncodedTaskMeta: encodedTask, + PartitionKeys: hashColPb, + Child: child, + Types: hashColTypes, + AllFieldTypes: allFieldTypes, + Compression: e.CompressionMode.ToTipbCompressionMode(), + UpstreamCteTaskMeta: encodedUpstreamCTETask, } executorID := e.ExplainID().String() return &tipb.Executor{ diff --git a/planner/core/stringer.go b/planner/core/stringer.go index 6d185146d414f..a383871aeef15 100644 --- a/planner/core/stringer.go +++ b/planner/core/stringer.go @@ -22,6 +22,51 @@ import ( "github.com/pingcap/tidb/util/plancodec" ) +func ToStringNewForMPP(p PhysicalPlan) string { + builder := &strings.Builder{} + childLen := len(p.Children()) + if childLen == 1 { + if _, ok := p.(*PhysicalExchangeReceiver); !ok { + fmt.Fprintf(builder, ToStringNewForMPP(p.Children()[0])) + fmt.Fprintf(builder, "->") + } + } else if childLen > 1 { + fmt.Fprintf(builder, "(") + for i := 0; i < childLen; i++ { + fmt.Fprintf(builder, ToStringNewForMPP(p.Children()[i])) + if i != childLen-1 { + fmt.Fprintf(builder, ",") + } + } + fmt.Fprintf(builder, ")->") + } + fmt.Fprintf(builder, p.TP()) + switch x := p.(type) { + case *PhysicalExchangeSender: + fmt.Fprintf(builder, "_%d", x.id) + fmt.Fprintf(builder, "(") + for _, task := range x.TargetTasks { + fmt.Fprintf(builder, "%d, ", task.ID) + } + for _, tasks := range x.TargetCTEReaderTasks { + fmt.Fprintf(builder, "(") + for _, task := range tasks { + fmt.Fprintf(builder, "%d, ", task.ID) + } + fmt.Fprintf(builder, ")") + } + fmt.Fprintf(builder, ")") + case *PhysicalExchangeReceiver: + fmt.Fprintf(builder, "_%d", x.id) + fmt.Fprintf(builder, "(") + for _, task := range x.Tasks { + fmt.Fprintf(builder, "%d, ", task.ID) + } + fmt.Fprintf(builder, ")") + } + return builder.String() +} + // ToString explains a Plan, returns description string. func ToString(p Plan) string { strs, _ := toString(p, []string{}, []int{}) @@ -358,6 +403,17 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { for _, task := range x.TargetTasks { str += fmt.Sprintf("%d, ", task.ID) } + for _, tasks := range x.TargetCTEReaderTasks { + str += "(" + for _, task := range tasks { + str += fmt.Sprintf("%d, ", task.ID) + } + str += ")" + } + str += ")" + case *PhysicalCTE: + str = "CTEReader(" + str += fmt.Sprintf("%v", x.CTE.IDForStorage) str += ")" default: str = fmt.Sprintf("%T", in) From 6044c0851f2f648f478d07873c4934fe5d1d2b38 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 1 Mar 2023 16:55:27 +0000 Subject: [PATCH 04/24] fix panics --- executor/adapter.go | 6 ++++++ planner/core/exhaust_physical_plans.go | 1 + planner/core/logical_plan_builder.go | 3 ++- planner/core/physical_plans.go | 4 ++++ 4 files changed, 13 insertions(+), 1 deletion(-) diff --git a/executor/adapter.go b/executor/adapter.go index 7a4649ed3f8b5..870edaaf1958a 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1174,6 +1174,12 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { } b := newExecutorBuilder(ctx, a.InfoSchema, a.Ti) + if !a.Ctx.GetSessionVars().InRestrictedSQL { + logutil.BgLogger().Warn("bulding executor", + zap.String("the top one's output cols", a.Plan.Schema().String()), + zap.String("the output names", fmt.Sprintf("%v", a.OutputNames)), + ) + } e := b.build(a.Plan) if b.err != nil { return nil, errors.Trace(b.err) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 8a7b7fcc22143..ce6b3d4873dc8 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -3293,6 +3293,7 @@ func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) } childReqs = append(childReqs, propChoice) seq := PhysicalSequence{}.Init(p.ctx, p.stats, p.blockOffset, childReqs...) + seq.SetSchema(p.children[len(p.children)-1].Schema()) seqs = append(seqs, seq) } return seqs, true, nil diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 45181666a6a7b..3c4b5f7cabf2a 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4261,7 +4261,7 @@ func (b *PlanBuilder) tryToBuildSequence(ctes []*cteInfo, p LogicalPlan) Logical if !ctes[i].nonRecursive { return p } - if ctes[i].isInline { + if ctes[i].isInline || ctes[i].cteClass == nil { ctes = append(ctes[:i], ctes[i+1:]...) } } @@ -4283,6 +4283,7 @@ func (b *PlanBuilder) tryToBuildSequence(ctes []*cteInfo, p LogicalPlan) Logical ctes: ctes, }.Init(b.ctx, b.getSelectOffset()) seq.SetChildren(append(lctes, p)...) + seq.SetOutputNames(p.OutputNames().Shallow()) return seq } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 8b0afc946e72f..f53508d8ed427 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -2645,3 +2645,7 @@ func (p *PhysicalSequence) Clone() (PhysicalPlan, error) { cloned.physicalSchemaProducer = *base return cloned, nil } + +func (p *PhysicalSequence) Schema() *expression.Schema { + return p.Children()[len(p.Children())-1].Schema() +} From 6c69b8a0dc1f881d273a1d3dd5a913238f711411 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 8 Mar 2023 21:36:54 +0800 Subject: [PATCH 05/24] some codes updates --- executor/mpp_gather.go | 81 ++++++++++++++++++++++++++ go.mod | 2 +- go.sum | 4 +- planner/core/exhaust_physical_plans.go | 30 +++++----- planner/core/find_best_task.go | 13 +++-- planner/core/flat_plan.go | 5 +- planner/core/fragment.go | 52 +++++++++++++++-- planner/core/logical_plan_builder.go | 1 + planner/core/physical_plans.go | 4 ++ planner/core/plan_cost_ver2.go | 15 +++++ planner/core/plan_to_pb.go | 5 ++ planner/core/stats.go | 6 ++ store/copr/mpp.go | 1 + 13 files changed, 192 insertions(+), 27 deletions(-) diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 6c67dfb8ee1db..d9a859b771740 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -16,10 +16,12 @@ package executor import ( "context" + "fmt" "time" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -109,6 +111,10 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { if err != nil { return errors.Trace(err) } + err = e.fixTaskForCTEStorageAndReader(dagReq.RootExecutor, mppTask.Meta) + if err != nil { + return err + } pbData, err := dagReq.Marshal() if err != nil { return errors.Trace(err) @@ -137,6 +143,81 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { return nil } +// fixTaskForCTEStorageAndReader fixes the upstream/downstream tasks for the producers and consumers. +// We only setup local transport for the data. +func (e *MPPGather) fixTaskForCTEStorageAndReader(exec *tipb.Executor, meta kv.MPPTaskMeta) error { + children := make([]*tipb.Executor, 0, 2) + switch exec.Tp { + case tipb.ExecType_TypeTableScan, tipb.ExecType_TypePartitionTableScan, tipb.ExecType_TypeIndexScan: + case tipb.ExecType_TypeSelection: + children = append(children, exec.Selection.Child) + case tipb.ExecType_TypeAggregation, tipb.ExecType_TypeStreamAgg: + children = append(children, exec.Aggregation.Child) + case tipb.ExecType_TypeTopN: + children = append(children, exec.TopN.Child) + case tipb.ExecType_TypeLimit: + children = append(children, exec.Limit.Child) + case tipb.ExecType_TypeExchangeSender: + children = append(children, exec.ExchangeSender.Child) + if len(exec.ExchangeSender.UpstreamCteTaskMeta) == 0 { + break + } + actualUpStreamTasks := make([][]byte, 0, len(exec.ExchangeSender.UpstreamCteTaskMeta)) + actualTIDs := make([]int64, 0, len(exec.ExchangeSender.UpstreamCteTaskMeta)) + for _, tasksFromOneConsumer := range exec.ExchangeSender.UpstreamCteTaskMeta { + for _, taskBytes := range tasksFromOneConsumer.EncodedTasks { + taskMeta := &mpp.TaskMeta{} + err := taskMeta.Unmarshal(taskBytes) + if err != nil { + return err + } + if taskMeta.Address != meta.GetAddress() { + continue + } + actualUpStreamTasks = append(actualUpStreamTasks, taskBytes) + actualTIDs = append(actualTIDs, taskMeta.TaskId) + } + } + logutil.BgLogger().Warn("refine tunnel for cte producer task", zap.String("the final tunnel", fmt.Sprintf("up stream consumer tasks: %v", actualTIDs))) + exec.ExchangeSender.EncodedTaskMeta = actualUpStreamTasks + case tipb.ExecType_TypeExchangeReceiver: + if len(exec.ExchangeReceiver.OriginalCtePrdocuerTaskMeta) == 0 { + break + } + exec.ExchangeReceiver.EncodedTaskMeta = [][]byte{} + actualTIDs := make([]int64, 0, 4) + for _, taskBytes := range exec.ExchangeReceiver.OriginalCtePrdocuerTaskMeta { + taskMeta := &mpp.TaskMeta{} + err := taskMeta.Unmarshal(taskBytes) + if err != nil { + return err + } + if taskMeta.Address != meta.GetAddress() { + continue + } + exec.ExchangeReceiver.EncodedTaskMeta = append(exec.ExchangeReceiver.EncodedTaskMeta, taskBytes) + actualTIDs = append(actualTIDs, taskMeta.TaskId) + } + logutil.BgLogger().Warn("refine tunnel for cte consumer task", zap.String("the final tunnel", fmt.Sprintf("down stream producer task: %v", actualTIDs))) + case tipb.ExecType_TypeJoin: + children = append(children, exec.Join.Children...) + case tipb.ExecType_TypeProjection: + children = append(children, exec.Projection.Child) + case tipb.ExecType_TypeWindow: + children = append(children, exec.Window.Child) + case tipb.ExecType_TypeSort: + children = append(children, exec.Sort.Child) + case tipb.ExecType_TypeExpand: + children = append(children, exec.Expand.Child) + default: + return errors.Errorf("unknown new tipb protocol %d", exec.Tp) + } + for _, child := range children { + e.fixTaskForCTEStorageAndReader(child, meta) + } + return nil +} + func collectPlanIDS(plan plannercore.PhysicalPlan, ids []int) []int { ids = append(ids, plan.ID()) for _, child := range plan.Children() { diff --git a/go.mod b/go.mod index 65034c8b88988..ed556778c4866 100644 --- a/go.mod +++ b/go.mod @@ -277,6 +277,6 @@ replace ( // fix potential security issue(CVE-2020-26160) introduced by indirect dependency. github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible github.com/pingcap/tidb/parser => ./parser - github.com/pingcap/tipb => github.com/pingcap/tipb v0.0.0-20230301075649-6f61c6d79ef3 + github.com/pingcap/tipb => github.com/pingcap/tipb v0.0.0-20230305184922-3290e9e4ca8a go.opencensus.io => go.opencensus.io v0.23.1-0.20220331163232-052120675fac ) diff --git a/go.sum b/go.sum index 37e0e88a1f28b..1a5caa8f6fa96 100644 --- a/go.sum +++ b/go.sum @@ -781,8 +781,8 @@ github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c h1:crhkw6DD+07Bg1wYh github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops= -github.com/pingcap/tipb v0.0.0-20230301075649-6f61c6d79ef3 h1:pmrwFxfD9t3wvXHJ5z9+UPZX5pc/jtEVemDFV8Sy3/E= -github.com/pingcap/tipb v0.0.0-20230301075649-6f61c6d79ef3/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20230305184922-3290e9e4ca8a h1:UV88ixc+osXKY+qa7fBellKgISQIOGtuqQuEAZ0zWVU= +github.com/pingcap/tipb v0.0.0-20230305184922-3290e9e4ca8a/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index ce6b3d4873dc8..434e9388a554c 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2260,8 +2260,8 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC lPartitionKeys = choosePartitionKeys(lPartitionKeys, matches) rPartitionKeys = choosePartitionKeys(rPartitionKeys, matches) } - childrenProps[0] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: lPartitionKeys, CanAddEnforcer: true, RejectSort: true} - childrenProps[1] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: rPartitionKeys, CanAddEnforcer: true, RejectSort: true} + childrenProps[0] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: lPartitionKeys, CanAddEnforcer: true, RejectSort: true, CTECanMPP: prop.CTECanMPP} + childrenProps[1] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: rPartitionKeys, CanAddEnforcer: true, RejectSort: true, CTECanMPP: prop.CTECanMPP} } join := PhysicalHashJoin{ basePhysicalJoin: baseJoin, @@ -2923,7 +2923,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // 2-phase agg // no partition property down,record partition cols inside agg itself, enforce shuffler latter. - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, RejectSort: true} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, RejectSort: true, CTECanMPP: prop.CTECanMPP} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = Mpp2Phase @@ -2932,7 +2932,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // agg runs on TiDB with a partial agg on TiFlash if possible if prop.TaskTp == property.RootTaskType { - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTECanMPP: prop.CTECanMPP} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = MppTiDB @@ -2940,7 +2940,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert } } else if !hasFinalAgg { // TODO: support scalar agg in MPP, merge the final result to one node - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTECanMPP: prop.CTECanMPP} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) if la.HasDistinct() || la.HasOrderBy() { @@ -3270,28 +3270,32 @@ func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) func (p *LogicalCTE) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { pcte := PhysicalCTE{CTE: p.cte}.Init(p.ctx, p.stats) + pcte.SetSchema(p.schema) pcte.childrenReqProps = []*property.PhysicalProperty{prop.CloneEssentialFields()} return []PhysicalPlan{(*PhysicalCTEStorage)(pcte)}, true, nil } func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { - possibleChildrenProps := make([]*property.PhysicalProperty, 0, 2) + possibleChildrenProps := make([][]*property.PhysicalProperty, 0, 2) childProp := prop.CloneEssentialFields() - possibleChildrenProps = append(possibleChildrenProps, childProp) + anyType := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, CanAddEnforcer: true, RejectSort: true, CTECanMPP: prop.CTECanMPP} + if prop.TaskTp == property.MppTaskType { + possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType.CloneEssentialFields(), childProp}) + } else { + possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{{TaskTp: property.RootTaskType, ExpectedCnt: math.MaxInt64}, childProp}) + } if prop.TaskTp != property.MppTaskType && - p.SCtx().GetSessionVars().IsMPPAllowed() { - childPropMpp := prop.CloneEssentialFields() - childPropMpp.TaskTp = property.MppTaskType - possibleChildrenProps = append(possibleChildrenProps, childPropMpp) + p.SCtx().GetSessionVars().IsMPPAllowed() && prop.IsSortItemEmpty() { + possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType.CloneEssentialFields(), anyType.CloneEssentialFields()}) } seqs := make([]PhysicalPlan, 0, 2) for _, propChoice := range possibleChildrenProps { childReqs := make([]*property.PhysicalProperty, 0, len(p.children)) for i := 0; i < len(p.children)-1; i++ { - childReqs = append(childReqs, propChoice) + childReqs = append(childReqs, propChoice[0].CloneEssentialFields()) } - childReqs = append(childReqs, propChoice) + childReqs = append(childReqs, propChoice[1]) seq := PhysicalSequence{}.Init(p.ctx, p.stats, p.blockOffset, childReqs...) seq.SetSchema(p.children[len(p.children)-1].Schema()) seqs = append(seqs, seq) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 092918a506a14..52f8d61c19e45 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -329,6 +329,11 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl _, isMpp := childTask.(*mppTask) allMpp = allMpp && isMpp } + // This check makes sure that there is no invalid child task. + if len(childTasks) != len(p.children)-1 { + continue + } + lastChildProp := pp.GetChildReqProps(lastIdx).CloneEssentialFields() lastChildProp.CTECanMPP = allMpp lastChildTask, cnt, err := p.Children()[lastIdx].findBestTask(lastChildProp, &PlanCounterDisabled, opt) @@ -338,15 +343,15 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } curCntPlan = curCntPlan * cnt if lastChildTask != nil && lastChildTask.invalid() { - break + continue } - childTasks = append(childTasks, lastChildTask) - // This check makes sure that there is no invalid child task. - if len(childTasks) != len(p.children) { + if _, ok := lastChildTask.(*mppTask); !ok && allMpp { continue } + childTasks = append(childTasks, lastChildTask) + // If the target plan can be found in this physicalPlan(pp), rebuild childTasks to build the corresponding combination. if planCounter.IsForce() && int64(*planCounter) <= curCntPlan { p.ctx.GetSessionVars().PlanID = savedPlanID diff --git a/planner/core/flat_plan.go b/planner/core/flat_plan.go index 328418f67f663..5efef86a84f43 100644 --- a/planner/core/flat_plan.go +++ b/planner/core/flat_plan.go @@ -356,7 +356,10 @@ func (f *FlatPhysicalPlan) flattenRecursively(p Plan, info *operatorCtx, target // for details) to affect the row count display of the independent CTE plan tree. copiedCTE := *plan copiedCTE.probeParents = nil - f.ctesToFlatten = append(f.ctesToFlatten, &copiedCTE) + if info.isRoot { + // If it's executed in TiDB, we need to record it since we don't have producer and consumer + f.ctesToFlatten = append(f.ctesToFlatten, &copiedCTE) + } case *Insert: if plan.SelectPlan != nil { childCtx.isRoot = true diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 88777f776cac9..9d7f6f20694e7 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" @@ -157,13 +158,24 @@ func (m *mppAddr) GetAddress() string { // for the task without table scan, we construct tasks according to the children's tasks. // That's for avoiding assigning to the failed node repeatly. We assumes that the chilren node must be workable. -func (e *mppTaskGenerator) constructMPPTasksByChildrenTasks(tasks []*kv.MPPTask) []*kv.MPPTask { +func (e *mppTaskGenerator) constructMPPTasksByChildrenTasks(tasks []*kv.MPPTask, cteProducerTasks []*kv.MPPTask) []*kv.MPPTask { addressMap := make(map[string]struct{}) newTasks := make([]*kv.MPPTask, 0, len(tasks)) + cteAddrMap := make(map[string]struct{}) + for _, task := range cteProducerTasks { + addr := task.Meta.GetAddress() + if _, ok := cteAddrMap[addr]; !ok { + cteAddrMap[addr] = struct{}{} + } + } for _, task := range tasks { addr := task.Meta.GetAddress() // for upper fragment, the task num is equal to address num covered by lower tasks _, ok := addressMap[addr] + if _, okk := cteAddrMap[addr]; !okk && len(cteAddrMap) > 0 { + // If we have cte producer, we reject other possible addresses. + continue + } if !ok { mppTask := &kv.MPPTask{ Meta: &mppAddr{addr: addr}, @@ -217,6 +229,9 @@ func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, fo cur := stack[len(stack)-1] switch x := cur.(type) { case *PhysicalTableScan, *PhysicalExchangeReceiver, *PhysicalCTE: // This should be the leave node. + if x.ID() == 359 { + logutil.BgLogger().Warn("1") + } p, err := stack[0].Clone() if err != nil { return errors.Trace(err) @@ -353,13 +368,19 @@ func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv for _, r := range f.ExchangeReceivers { childrenTasks = append(childrenTasks, r.Tasks...) } + cteProducerTasks := make([]*kv.MPPTask, 0) for _, cteR := range f.CTEReaders { - childrenTasks = append(childrenTasks, cteR.children[0].(*PhysicalExchangeReceiver).Tasks...) + child := cteR.children[0] + if _, ok := child.(*PhysicalProjection); ok { + child = child.Children()[0] + } + cteProducerTasks = append(cteProducerTasks, child.(*PhysicalExchangeReceiver).Tasks...) + childrenTasks = append(childrenTasks, child.(*PhysicalExchangeReceiver).Tasks...) } if f.singleton && len(childrenTasks) > 0 { childrenTasks = childrenTasks[0:1] } - tasks = e.constructMPPTasksByChildrenTasks(childrenTasks) + tasks = e.constructMPPTasksByChildrenTasks(childrenTasks, cteProducerTasks) } if err != nil { return nil, errors.Trace(err) @@ -384,8 +405,6 @@ func (f *Fragment) flipCTEReader(currentPlan PhysicalPlan) { newChildren[i] = child if cteR, ok := child.(*PhysicalCTE); ok { receiver := cteR.Children()[0] - receiver.SetChildren(cteR) - cteR.SetChildren(nil) newChildren[i] = receiver } else if _, ok := child.(*PhysicalExchangeReceiver); !ok { // The receiver is the leaf of the fragment though it has child, we need break it. @@ -396,6 +415,9 @@ func (f *Fragment) flipCTEReader(currentPlan PhysicalPlan) { } func (e *mppTaskGenerator) generateTasksForCTEReader(cteReader *PhysicalCTE) (err error) { + if cteReader.CTE.IDForStorage == 2 { + logutil.BgLogger().Warn("1") + } group := e.CTEGroups[cteReader.CTE.IDForStorage] if group.StroageFragments == nil { exchangeSender := PhysicalExchangeSender{ @@ -407,10 +429,28 @@ func (e *mppTaskGenerator) generateTasksForCTEReader(cteReader *PhysicalCTE) (er return err } } - receiver := PhysicalExchangeReceiver{}.Init(cteReader.SCtx(), group.CTEStorage.statsInfo()) + receiver := PhysicalExchangeReceiver{IsCTEReader: true}.Init(cteReader.SCtx(), group.CTEStorage.statsInfo()) receiver.Tasks = group.StroageTasks receiver.frags = group.StroageFragments cteReader.SetChildren(receiver) + receiver.SetChildren(group.CTEStorage.children[0]) + inconsistenceNullable := false + for i, col := range cteReader.schema.Columns { + if mysql.HasNotNullFlag(col.RetType.GetFlag()) != mysql.HasNotNullFlag(group.CTEStorage.children[0].Schema().Columns[i].RetType.GetFlag()) { + inconsistenceNullable = true + break + } + } + if inconsistenceNullable { + cols := group.CTEStorage.children[0].Schema().Clone().Columns + for i, col := range cols { + col.Index = i + } + proj := PhysicalProjection{Exprs: expression.Column2Exprs(cols)}.Init(cteReader.ctx, cteReader.stats, 0, nil) + proj.SetSchema(cteReader.schema.Clone()) + proj.SetChildren(receiver) + cteReader.SetChildren(proj) + } return nil } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 3c4b5f7cabf2a..cd0a8c8a848be 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4276,6 +4276,7 @@ func (b *PlanBuilder) tryToBuildSequence(ctes []*cteInfo, p LogicalPlan) Logical cteName: cte.def.Name, seedStat: cte.seedStat, }.Init(b.ctx, b.getSelectOffset()) + lcte.SetSchema(getResultCTESchema(cte.seedLP.Schema(), b.ctx.GetSessionVars())) lctes = append(lctes, lcte) lcte.SetChildren(cte.seedLP) } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index f53508d8ed427..f276ecd44153f 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1478,6 +1478,8 @@ type PhysicalExchangeReceiver struct { Tasks []*kv.MPPTask frags []*Fragment + + IsCTEReader bool } // Clone implment PhysicalPlan interface. @@ -1488,6 +1490,8 @@ func (p *PhysicalExchangeReceiver) Clone() (PhysicalPlan, error) { return nil, errors.Trace(err) } np.basePhysicalPlan = *base + + np.IsCTEReader = p.IsCTEReader return np, nil } diff --git a/planner/core/plan_cost_ver2.go b/planner/core/plan_cost_ver2.go index 1f1569e0c14df..84f479ff4beca 100644 --- a/planner/core/plan_cost_ver2.go +++ b/planner/core/plan_cost_ver2.go @@ -748,6 +748,21 @@ func (p *BatchPointGetPlan) getPlanCostVer2(taskType property.TaskType, option * return p.planCostVer2, nil } +func (p *PhysicalCTE) getPlanCostVer2(taskType property.TaskType, option *PlanCostOption) (costVer2, error) { + if p.planCostInit && !hasCostFlag(option.CostFlag, CostFlagRecalculate) { + return p.planCostVer2, nil + } + + inputRows := getCardinality(p, option.CostFlag) + cpuFactor := getTaskCPUFactorVer2(p, taskType) + + projCost := filterCostVer2(option, inputRows, expression.Column2Exprs(p.schema.Columns), cpuFactor) + + p.planCostVer2 = projCost + p.planCostInit = true + return p.planCostVer2, nil +} + func scanCostVer2(option *PlanCostOption, rows, rowSize float64, scanFactor costVer2Factor) costVer2 { if rowSize < 1 { rowSize = 1 diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index f3ecf7734ddb1..0edd4a4b177c6 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -384,6 +384,11 @@ func (e *PhysicalExchangeReceiver) ToPB(ctx sessionctx.Context, _ kv.StoreType) EncodedTaskMeta: encodedTask, FieldTypes: fieldTypes, } + if e.IsCTEReader { + encodedTaskShallowCopy := make([][]byte, len(e.Tasks)) + copy(encodedTaskShallowCopy, encodedTask) + ecExec.OriginalCtePrdocuerTaskMeta = encodedTaskShallowCopy + } executorID := e.ExplainID().String() return &tipb.Executor{ Tp: tipb.ExecType_TypeExchangeReceiver, diff --git a/planner/core/stats.go b/planner/core/stats.go index 081ec859d492a..b3c3051660900 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -1063,6 +1063,12 @@ func (p *LogicalCTE) recursiveDeriveStats(colGroups [][]*expression.Column) (*pr return p.DeriveStats(nil, p.self.Schema(), nil, colGroups) } var err error + if len(p.cte.pushDownPredicates) > 0 { + newCond := expression.ComposeDNFCondition(p.ctx, p.cte.pushDownPredicates...) + sel := LogicalSelection{Conditions: []expression.Expression{newCond}}.Init(p.ctx, p.blockOffset) + sel.SetChildren(p.children[0]) + p.SetChildren(sel) + } p.children[0], err = logicalOptimize(context.TODO(), p.cte.optFlag, p.children[0]) if err != nil { return nil, err diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 3b4efc96b2c2d..512a6a2680af7 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -492,6 +492,7 @@ func (m *mppIterator) handleMPPStreamResponse(bo *Backoffer, response *mpp.MPPDa zap.Uint64("txnStartTS", req.StartTs), zap.String("storeAddr", req.Meta.GetAddress()), zap.Int64("mpp-version", m.mppVersion.ToInt64()), + zap.Int64("task id", req.ID), zap.Error(err)) return err } From d6c27bfbd0cd38bf0af1764e241ad665766ccd41 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 9 Mar 2023 04:26:18 +0800 Subject: [PATCH 06/24] update the codes --- planner/core/exhaust_physical_plans.go | 41 ++++++++++++++------------ planner/core/find_best_task.go | 22 +++++++------- planner/property/physical_property.go | 19 +++++++----- sessionctx/variable/session.go | 3 -- 4 files changed, 44 insertions(+), 41 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 434e9388a554c..cefe7bfd722fa 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2213,7 +2213,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC baseJoin.InnerChildIdx = preferredBuildIndex childrenProps := make([]*property.PhysicalProperty, 2) if useBCJ { - childrenProps[preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.BroadcastType, CanAddEnforcer: true, RejectSort: true, CTECanMPP: prop.CTECanMPP} + childrenProps[preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.BroadcastType, CanAddEnforcer: true, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} expCnt := math.MaxFloat64 if prop.ExpectedCnt < p.stats.RowCount { expCntScale := prop.ExpectedCnt / p.stats.RowCount @@ -2226,12 +2226,12 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC hashKeys = lPartitionKeys } if matches := prop.IsSubsetOf(hashKeys); len(matches) != 0 { - childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.HashType, MPPPartitionCols: prop.MPPPartitionCols, RejectSort: true, CTECanMPP: prop.CTECanMPP} + childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.HashType, MPPPartitionCols: prop.MPPPartitionCols, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} } else { return nil } } else { - childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.AnyType, RejectSort: true, CTECanMPP: prop.CTECanMPP} + childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.AnyType, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} } } else { lPartitionKeys, rPartitionKeys := p.GetPotentialPartitionKeys() @@ -2260,8 +2260,8 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC lPartitionKeys = choosePartitionKeys(lPartitionKeys, matches) rPartitionKeys = choosePartitionKeys(rPartitionKeys, matches) } - childrenProps[0] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: lPartitionKeys, CanAddEnforcer: true, RejectSort: true, CTECanMPP: prop.CTECanMPP} - childrenProps[1] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: rPartitionKeys, CanAddEnforcer: true, RejectSort: true, CTECanMPP: prop.CTECanMPP} + childrenProps[0] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: lPartitionKeys, CanAddEnforcer: true, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} + childrenProps[1] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: rPartitionKeys, CanAddEnforcer: true, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} } join := PhysicalHashJoin{ basePhysicalJoin: baseJoin, @@ -2375,7 +2375,7 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { - resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: math.MaxFloat64, CTECanMPP: prop.CTECanMPP} + resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: math.MaxFloat64, CTEConsumerStatus: prop.CTEConsumerStatus} topN := PhysicalTopN{ ByItems: lt.ByItems, PartitionBy: lt.PartitionBy, @@ -2552,7 +2552,7 @@ func (lw *LogicalWindow) tryToGetMppWindows(prop *property.PhysicalProperty) []P SortItems: byItems, TaskTp: property.MppTaskType, SortItemsForPartition: byItems, - CTECanMPP: prop.CTECanMPP, + CTEConsumerStatus: prop.CTEConsumerStatus, } if !prop.IsPrefix(childProperty) { return nil @@ -2909,7 +2909,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // To avoid mess, we don't do any one-phase aggregation in this case. // If this is a skew distinct group agg, skip generating 1-phase agg, because skew data will cause performance issue if len(partitionCols) != 0 && !la.ctx.GetSessionVars().EnableSkewDistinctAgg { - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: partitionCols, CanAddEnforcer: true, RejectSort: true, CTECanMPP: prop.CTECanMPP} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: partitionCols, CanAddEnforcer: true, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = Mpp1Phase @@ -2923,7 +2923,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // 2-phase agg // no partition property down,record partition cols inside agg itself, enforce shuffler latter. - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, RejectSort: true, CTECanMPP: prop.CTECanMPP} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = Mpp2Phase @@ -2932,7 +2932,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // agg runs on TiDB with a partial agg on TiFlash if possible if prop.TaskTp == property.RootTaskType { - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTECanMPP: prop.CTECanMPP} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = MppTiDB @@ -2940,7 +2940,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert } } else if !hasFinalAgg { // TODO: support scalar agg in MPP, merge the final result to one node - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTECanMPP: prop.CTECanMPP} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) if la.HasDistinct() || la.HasOrderBy() { @@ -3138,7 +3138,7 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { - resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(p.Count + p.Offset), CTECanMPP: prop.CTECanMPP} + resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(p.Count + p.Offset), CTEConsumerStatus: prop.CTEConsumerStatus} limit := PhysicalLimit{ Offset: p.Offset, Count: p.Count, @@ -3179,10 +3179,10 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) for range p.children { if canUseMpp && prop.TaskTp == property.MppTaskType { chReqProps = append(chReqProps, &property.PhysicalProperty{ - ExpectedCnt: prop.ExpectedCnt, - TaskTp: property.MppTaskType, - RejectSort: true, - CTECanMPP: prop.CTECanMPP, + ExpectedCnt: prop.ExpectedCnt, + TaskTp: property.MppTaskType, + RejectSort: true, + CTEConsumerStatus: prop.CTEConsumerStatus, }) } else { chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: prop.ExpectedCnt, RejectSort: true}) @@ -3220,7 +3220,7 @@ func (p *LogicalPartitionUnionAll) exhaustPhysicalPlans(prop *property.PhysicalP } func (ls *LogicalSort) getPhysicalSort(prop *property.PhysicalProperty) *PhysicalSort { - ps := PhysicalSort{ByItems: ls.ByItems}.Init(ls.ctx, ls.stats.ScaleByExpectCnt(prop.ExpectedCnt), ls.blockOffset, &property.PhysicalProperty{TaskTp: prop.TaskTp, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTECanMPP: prop.CTECanMPP}) + ps := PhysicalSort{ByItems: ls.ByItems}.Init(ls.ctx, ls.stats.ScaleByExpectCnt(prop.ExpectedCnt), ls.blockOffset, &property.PhysicalProperty{TaskTp: prop.TaskTp, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus}) return ps } @@ -3278,14 +3278,17 @@ func (p *LogicalCTE) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]Ph func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { possibleChildrenProps := make([][]*property.PhysicalProperty, 0, 2) childProp := prop.CloneEssentialFields() - anyType := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, CanAddEnforcer: true, RejectSort: true, CTECanMPP: prop.CTECanMPP} + anyType := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, CanAddEnforcer: true, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} if prop.TaskTp == property.MppTaskType { + if prop.CTEConsumerStatus == property.SomeCTEFailedMpp { + return nil, true, nil + } possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType.CloneEssentialFields(), childProp}) } else { possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{{TaskTp: property.RootTaskType, ExpectedCnt: math.MaxInt64}, childProp}) } - if prop.TaskTp != property.MppTaskType && + if prop.TaskTp != property.MppTaskType && prop.CTEConsumerStatus != property.SomeCTEFailedMpp && p.SCtx().GetSessionVars().IsMPPAllowed() && prop.IsSortItemEmpty() { possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType.CloneEssentialFields(), anyType.CloneEssentialFields()}) } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 52f8d61c19e45..ac96ac5459716 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -299,10 +299,6 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl childTasks := make([]task, 0, len(p.children)) childCnts := make([]int64, len(p.children)) cntPlan = 0 - origCTECanMPP := p.ctx.GetSessionVars().CurrentCTECanMPP - defer func() { - p.ctx.GetSessionVars().CurrentCTECanMPP = origCTECanMPP - }() for _, pp := range physicalPlans { // Find best child tasks firstly. childTasks = childTasks[:0] @@ -310,9 +306,7 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl curCntPlan = 1 timeStampNow := p.GetLogicalTS4TaskMap() savedPlanID := p.ctx.GetSessionVars().PlanID - allMpp := true lastIdx := len(p.children) - 1 - p.ctx.GetSessionVars().CurrentCTECanMPP = true for j := 0; j < lastIdx; j++ { child := p.children[j] childProp := pp.GetChildReqProps(j) @@ -325,9 +319,11 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl if childTask != nil && childTask.invalid() { break } - childTasks = append(childTasks, childTask) _, isMpp := childTask.(*mppTask) - allMpp = allMpp && isMpp + if !isMpp && prop.IsFlashProp() { + break + } + childTasks = append(childTasks, childTask) } // This check makes sure that there is no invalid child task. if len(childTasks) != len(p.children)-1 { @@ -335,7 +331,9 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } lastChildProp := pp.GetChildReqProps(lastIdx).CloneEssentialFields() - lastChildProp.CTECanMPP = allMpp + if lastChildProp.IsFlashProp() { + lastChildProp.CTEConsumerStatus = property.AllCTECanMpp + } lastChildTask, cnt, err := p.Children()[lastIdx].findBestTask(lastChildProp, &PlanCounterDisabled, opt) childCnts[lastIdx] = cnt if err != nil { @@ -346,7 +344,7 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl continue } - if _, ok := lastChildTask.(*mppTask); !ok && allMpp { + if _, ok := lastChildTask.(*mppTask); !ok && lastChildProp.CTEConsumerStatus == property.AllCTECanMpp { continue } @@ -2676,8 +2674,8 @@ func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *Plan // The physical plan has been build when derive stats. pcte := PhysicalCTE{SeedPlan: p.cte.seedPartPhysicalPlan, RecurPlan: p.cte.recursivePartPhysicalPlan, CTE: p.cte, cteAsName: p.cteAsName, cteName: p.cteName}.Init(p.ctx, p.stats) pcte.SetSchema(p.schema) - logutil.BgLogger().Warn("build cte", zap.String("prop type", prop.TaskTp.String()), zap.Bool("can mpp", prop.CTECanMPP), zap.String("cte name", pcte.ExplainID().String())) - if prop.IsFlashProp() && prop.CTECanMPP { + logutil.BgLogger().Warn("build cte", zap.String("prop type", prop.TaskTp.String()), zap.Bool("can mpp", prop.CTEConsumerStatus == property.AllCTECanMpp), zap.String("cte name", pcte.ExplainID().String())) + if prop.IsFlashProp() && prop.CTEConsumerStatus == property.AllCTECanMpp { if prop.MPPPartitionTp != property.AnyType { return invalidTask, 1, nil } diff --git a/planner/property/physical_property.go b/planner/property/physical_property.go index a57a935502d55..b7ee7ba6e8aea 100644 --- a/planner/property/physical_property.go +++ b/planner/property/physical_property.go @@ -161,6 +161,15 @@ func GetCollateNameByIDForPartition(collateID int32) string { return collate.CollationID2Name(collateID) } +// cteConsumerStatus indicates whether we can let the current CTE consumer/reader be executed on the MPP nodes. +type cteConsumerStatus int + +const ( + NoCTE cteConsumerStatus = iota + SomeCTEFailedMpp + AllCTECanMpp +) + // PhysicalProperty stands for the required physical property by parents. // It contains the orders and the task types. type PhysicalProperty struct { @@ -203,7 +212,7 @@ type PhysicalProperty struct { // Non-MPP tasks do not care about it. RejectSort bool - CTECanMPP bool + CTEConsumerStatus cteConsumerStatus } // NewPhysicalProperty builds property from columns. @@ -330,12 +339,8 @@ func (p *PhysicalProperty) HashCode() []byte { for _, col := range p.MPPPartitionCols { p.hashcode = append(p.hashcode, col.hashCode(nil)...) } - if p.CTECanMPP { - p.hashcode = append(p.hashcode, codec.EncodeInt(p.hashcode, 1)...) - } else { - p.hashcode = append(p.hashcode, codec.EncodeInt(p.hashcode, 0)...) - } } + p.hashcode = append(p.hashcode, codec.EncodeInt(nil, int64(p.CTEConsumerStatus))...) return p.hashcode } @@ -355,7 +360,7 @@ func (p *PhysicalProperty) CloneEssentialFields() *PhysicalProperty { MPPPartitionTp: p.MPPPartitionTp, MPPPartitionCols: p.MPPPartitionCols, RejectSort: p.RejectSort, - CTECanMPP: p.CTECanMPP, + CTEConsumerStatus: p.CTEConsumerStatus, } return prop } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index a0e57f1146cc7..117fb13d264d9 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -706,9 +706,6 @@ type SessionVars struct { // PlanColumnID is the unique id for column when building plan. PlanColumnID int64 - // CurrentCTECanMPP indicates we can make the current cte operator executed on MPP engine. - CurrentCTECanMPP bool - // MapHashCode2UniqueID4ExtendedCol map the expr's hash code to specified unique ID. MapHashCode2UniqueID4ExtendedCol map[string]int From 9133d92feeabd3d486dd93e562eeae7c5e95b4bf Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 15 Mar 2023 17:52:58 +0800 Subject: [PATCH 07/24] change style and clean --- distsql/select_result.go | 5 +- planner/core/exhaust_physical_plans.go | 6 +++ planner/core/find_best_task.go | 3 +- planner/core/fragment.go | 13 ++--- planner/core/logical_plan_builder.go | 10 ++-- planner/core/logical_plans.go | 2 + planner/core/optimizer.go | 24 +++++---- planner/core/physical_plans.go | 17 +++++++ planner/core/stats.go | 70 +++++++++++++------------- 9 files changed, 85 insertions(+), 65 deletions(-) diff --git a/distsql/select_result.go b/distsql/select_result.go index 394298e8fa3b0..c060a1dcbef4a 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -395,12 +395,11 @@ func (r *selectResult) updateCopRuntimeStats(ctx context.Context, copStats *copr } if hasExecutor { var recorededPlanIDs = make(map[int]int) - for i, detail := range r.selectResp.GetExecutionSummaries() { + for _, detail := range r.selectResp.GetExecutionSummaries() { if detail != nil && detail.TimeProcessedNs != nil && detail.NumProducedRows != nil && detail.NumIterations != nil { - planID := r.copPlanIDs[i] recorededPlanIDs[r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl. - RecordOneCopTask(planID, r.storeType.Name(), callee, detail)] = 0 + RecordOneCopTask(-1, r.storeType.Name(), callee, detail)] = 0 } } num := uint64(0) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index cefe7bfd722fa..860ed5a5d3f1b 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/size" + "github.com/pingcap/tipb/go-tipb" "go.uber.org/atomic" "go.uber.org/zap" "golang.org/x/exp/slices" @@ -3270,6 +3271,11 @@ func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) func (p *LogicalCTE) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { pcte := PhysicalCTE{CTE: p.cte}.Init(p.ctx, p.stats) + if prop.IsFlashProp() { + pcte.storageSender = PhysicalExchangeSender{ + ExchangeType: tipb.ExchangeType_Broadcast, + }.Init(p.ctx, p.stats) + } pcte.SetSchema(p.schema) pcte.childrenReqProps = []*property.PhysicalProperty{prop.CloneEssentialFields()} return []PhysicalPlan{(*PhysicalCTEStorage)(pcte)}, true, nil diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index ac96ac5459716..2aa3042001ea8 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -2665,7 +2665,6 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *PlanCounterTp, pop *physicalOptimizeOp) (t task, cntPlan int64, err error) { if len(p.children) > 0 { - logutil.BgLogger().Warn("build physical for cte", zap.String("logical plan", ToString(p.children[0]))) return p.baseLogicalPlan.findBestTask(prop, counter, pop) } if !prop.IsSortItemEmpty() && !prop.CanAddEnforcer { @@ -2674,8 +2673,8 @@ func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *Plan // The physical plan has been build when derive stats. pcte := PhysicalCTE{SeedPlan: p.cte.seedPartPhysicalPlan, RecurPlan: p.cte.recursivePartPhysicalPlan, CTE: p.cte, cteAsName: p.cteAsName, cteName: p.cteName}.Init(p.ctx, p.stats) pcte.SetSchema(p.schema) - logutil.BgLogger().Warn("build cte", zap.String("prop type", prop.TaskTp.String()), zap.Bool("can mpp", prop.CTEConsumerStatus == property.AllCTECanMpp), zap.String("cte name", pcte.ExplainID().String())) if prop.IsFlashProp() && prop.CTEConsumerStatus == property.AllCTECanMpp { + pcte.readerRecevier = PhysicalExchangeReceiver{IsCTEReader: true}.Init(p.ctx, p.stats) if prop.MPPPartitionTp != property.AnyType { return invalidTask, 1, nil } diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 9d7f6f20694e7..a3a3367f7701a 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -415,21 +414,15 @@ func (f *Fragment) flipCTEReader(currentPlan PhysicalPlan) { } func (e *mppTaskGenerator) generateTasksForCTEReader(cteReader *PhysicalCTE) (err error) { - if cteReader.CTE.IDForStorage == 2 { - logutil.BgLogger().Warn("1") - } group := e.CTEGroups[cteReader.CTE.IDForStorage] if group.StroageFragments == nil { - exchangeSender := PhysicalExchangeSender{ - ExchangeType: tipb.ExchangeType(property.BroadcastType), - }.Init(group.CTEStorage.SCtx(), group.CTEStorage.children[0].statsInfo()) - exchangeSender.SetChildren(group.CTEStorage.children[0]) - group.StroageTasks, group.StroageFragments, err = e.generateMPPTasksForExchangeSender(exchangeSender) + group.CTEStorage.storageSender.SetChildren(group.CTEStorage.children...) + group.StroageTasks, group.StroageFragments, err = e.generateMPPTasksForExchangeSender(group.CTEStorage.storageSender) if err != nil { return err } } - receiver := PhysicalExchangeReceiver{IsCTEReader: true}.Init(cteReader.SCtx(), group.CTEStorage.statsInfo()) + receiver := cteReader.readerRecevier receiver.Tasks = group.StroageTasks receiver.frags = group.StroageFragments cteReader.SetChildren(receiver) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index cd0a8c8a848be..57810ad43f778 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4271,14 +4271,14 @@ func (b *PlanBuilder) tryToBuildSequence(ctes []*cteInfo, p LogicalPlan) Logical lctes := make([]LogicalPlan, 0, len(ctes)+1) for _, cte := range ctes { lcte := LogicalCTE{ - cte: cte.cteClass, - cteAsName: cte.def.Name, - cteName: cte.def.Name, - seedStat: cte.seedStat, + cte: cte.cteClass, + cteAsName: cte.def.Name, + cteName: cte.def.Name, + seedStat: cte.seedStat, + onlyUsedAsStorage: true, }.Init(b.ctx, b.getSelectOffset()) lcte.SetSchema(getResultCTESchema(cte.seedLP.Schema(), b.ctx.GetSessionVars())) lctes = append(lctes, lcte) - lcte.SetChildren(cte.seedLP) } seq := LogicalSequence{ ctes: ctes, diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 7ce1e367946b0..544330c06fdb2 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -2014,6 +2014,8 @@ type LogicalCTE struct { cteName model.CIStr seedStat *property.StatsInfo isOuterMostCTE bool + + onlyUsedAsStorage bool } // LogicalCTETable is for CTE table diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 98ef2e55e02a3..9bb0b81110803 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -274,20 +274,16 @@ func checkStableResultMode(sctx sessionctx.Context) bool { return s.EnableStableResultMode && (!st.InInsertStmt && !st.InUpdateStmt && !st.InDeleteStmt && !st.InLoadDataStmt) } -// DoOptimize optimizes a logical plan to a physical plan. -func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (PhysicalPlan, float64, error) { +// DoOptimizeAndLogicAsRet optimizes a logical plan to a physical plan and return the optimized logical plan. +func DoOptimizeAndLogicAsRet(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (LogicalPlan, PhysicalPlan, float64, error) { logic, err := logicalOptimize(ctx, flag, logic) if err != nil { - return nil, 0, err - } - - if !sctx.GetSessionVars().InRestrictedSQL { - logutil.BgLogger().Warn("optimizing", zap.String("plan", ToString(logic))) + return nil, nil, 0, err } if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { - return nil, 0, errors.Trace(ErrCartesianProductUnsupported) + return nil, nil, 0, errors.Trace(ErrCartesianProductUnsupported) } planCounter := PlanCounterTp(sctx.GetSessionVars().StmtCtx.StmtHints.ForceNthPlan) if planCounter == 0 { @@ -295,11 +291,11 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic } physical, cost, err := physicalOptimize(logic, &planCounter) if err != nil { - return nil, 0, err + return nil, nil, 0, err } finalPlan, err := postOptimize(ctx, sctx, physical) if err != nil { - return nil, 0, err + return nil, nil, 0, err } if sctx.GetSessionVars().StmtCtx.EnableOptimizerCETrace { @@ -308,7 +304,13 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic if sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace { sctx.GetSessionVars().StmtCtx.OptimizeTracer.RecordFinalPlan(finalPlan.buildPlanTrace()) } - return finalPlan, cost, nil + return logic, finalPlan, cost, nil +} + +// DoOptimize optimizes a logical plan to a physical plan. +func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (PhysicalPlan, float64, error) { + _, finalPlan, cost, err := DoOptimizeAndLogicAsRet(ctx, sctx, flag, logic) + return finalPlan, cost, err } // refineCETrace will adjust the content of CETrace. diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index f276ecd44153f..c4f722db90c30 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -2424,6 +2424,9 @@ type PhysicalCTE struct { CTE *CTEClass cteAsName model.CIStr cteName model.CIStr + + readerRecevier *PhysicalExchangeReceiver + storageSender *PhysicalExchangeSender } // PhysicalCTETable is for CTE table. @@ -2483,6 +2486,20 @@ func (p *PhysicalCTE) Clone() (PhysicalPlan, error) { } cloned.cteAsName, cloned.cteName = p.cteAsName, p.cteName cloned.CTE = p.CTE + if p.storageSender != nil { + clonedSender, err := p.storageSender.Clone() + if err != nil { + return nil, err + } + cloned.storageSender = clonedSender.(*PhysicalExchangeSender) + } + if p.readerRecevier != nil { + clonedReceiver, err := p.readerRecevier.Clone() + if err != nil { + return nil, err + } + cloned.readerRecevier = clonedReceiver.(*PhysicalExchangeReceiver) + } return cloned, nil } diff --git a/planner/core/stats.go b/planner/core/stats.go index b3c3051660900..09e21f4407b47 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -116,6 +116,10 @@ func GetStats4Test(p LogicalPlan) *property.StatsInfo { } func (p *baseLogicalPlan) recursiveDeriveStats(colGroups [][]*expression.Column) (*property.StatsInfo, error) { + if len(p.self.Schema().Columns) == 0 { + logutil.BgLogger().Warn("xxxx", zap.String("the plan", fmt.Sprintf("%v", p.ExplainID())), zap.String("tree", ToString(p.self))) + return nil, errors.Errorf("?????") + } childStats := make([]*property.StatsInfo, len(p.children)) childSchema := make([]*expression.Schema, len(p.children)) cumColGroups := p.self.ExtractColGroups(colGroups) @@ -995,11 +999,6 @@ func (p *LogicalCTE) DeriveStats(childStats []*property.StatsInfo, selfSchema *e return p.stats, nil } - if len(p.children) > 0 { - p.stats = childStats[0] - return p.stats, nil - } - var err error if p.cte.seedPartPhysicalPlan == nil { // Build push-downed predicates. @@ -1009,11 +1008,14 @@ func (p *LogicalCTE) DeriveStats(childStats []*property.StatsInfo, selfSchema *e newSel.SetChildren(p.cte.seedPartLogicalPlan) p.cte.seedPartLogicalPlan = newSel } - p.cte.seedPartPhysicalPlan, _, err = DoOptimize(context.TODO(), p.ctx, p.cte.optFlag, p.cte.seedPartLogicalPlan) + p.cte.seedPartLogicalPlan, p.cte.seedPartPhysicalPlan, _, err = DoOptimizeAndLogicAsRet(context.TODO(), p.ctx, p.cte.optFlag, p.cte.seedPartLogicalPlan) if err != nil { return nil, err } } + if p.onlyUsedAsStorage { + p.SetChildren(p.cte.seedPartLogicalPlan) + } resStat := p.cte.seedPartPhysicalPlan.Stats() // Changing the pointer so that seedStat in LogicalCTETable can get the new stat. *p.seedStat = *resStat @@ -1058,31 +1060,31 @@ func (p *LogicalSequence) DeriveStats(childStats []*property.StatsInfo, _ *expre return p.stats, nil } -func (p *LogicalCTE) recursiveDeriveStats(colGroups [][]*expression.Column) (*property.StatsInfo, error) { - if len(p.children) == 0 { - return p.DeriveStats(nil, p.self.Schema(), nil, colGroups) - } - var err error - if len(p.cte.pushDownPredicates) > 0 { - newCond := expression.ComposeDNFCondition(p.ctx, p.cte.pushDownPredicates...) - sel := LogicalSelection{Conditions: []expression.Expression{newCond}}.Init(p.ctx, p.blockOffset) - sel.SetChildren(p.children[0]) - p.SetChildren(sel) - } - p.children[0], err = logicalOptimize(context.TODO(), p.cte.optFlag, p.children[0]) - if err != nil { - return nil, err - } - childStats := make([]*property.StatsInfo, len(p.children)) - childSchema := make([]*expression.Schema, len(p.children)) - cumColGroups := p.self.ExtractColGroups(colGroups) - for i, child := range p.children { - childProfile, err := child.recursiveDeriveStats(cumColGroups) - if err != nil { - return nil, err - } - childStats[i] = childProfile - childSchema[i] = child.Schema() - } - return p.self.DeriveStats(childStats, p.self.Schema(), childSchema, colGroups) -} +// func (p *LogicalCTE) recursiveDeriveStats(colGroups [][]*expression.Column) (*property.StatsInfo, error) { +// if len(p.children) == 0 { +// return p.DeriveStats(nil, p.self.Schema(), nil, colGroups) +// } +// var err error +// if len(p.cte.pushDownPredicates) > 0 { +// newCond := expression.ComposeDNFCondition(p.ctx, p.cte.pushDownPredicates...) +// sel := LogicalSelection{Conditions: []expression.Expression{newCond}}.Init(p.ctx, p.blockOffset) +// sel.SetChildren(p.children[0]) +// p.SetChildren(sel) +// } +// p.children[0], err = logicalOptimize(context.TODO(), p.cte.optFlag, p.children[0]) +// if err != nil { +// return nil, err +// } +// childStats := make([]*property.StatsInfo, len(p.children)) +// childSchema := make([]*expression.Schema, len(p.children)) +// cumColGroups := p.self.ExtractColGroups(colGroups) +// for i, child := range p.children { +// childProfile, err := child.recursiveDeriveStats(cumColGroups) +// if err != nil { +// return nil, err +// } +// childStats[i] = childProfile +// childSchema[i] = child.Schema() +// } +// return p.self.DeriveStats(childStats, p.self.Schema(), childSchema, colGroups) +// } From 6c81152e7d8f8a8b3a26923322bab9b3432d969a Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 15 Mar 2023 18:12:08 +0800 Subject: [PATCH 08/24] clean the debugging info, make it ready for review --- executor/adapter.go | 6 ---- executor/mpp_gather.go | 11 ------- planner/core/exhaust_physical_plans.go | 10 +----- planner/core/find_best_task.go | 3 -- planner/core/fragment.go | 5 --- planner/core/logical_plan_builder.go | 4 +-- planner/core/logical_plans.go | 2 -- planner/core/planbuilder.go | 5 --- planner/core/stats.go | 33 ------------------- planner/core/stringer.go | 45 -------------------------- 10 files changed, 2 insertions(+), 122 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 5ad80199ec7da..4c4f227559aaa 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1165,12 +1165,6 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { } b := newExecutorBuilder(ctx, a.InfoSchema, a.Ti) - if !a.Ctx.GetSessionVars().InRestrictedSQL { - logutil.BgLogger().Warn("bulding executor", - zap.String("the top one's output cols", a.Plan.Schema().String()), - zap.String("the output names", fmt.Sprintf("%v", a.OutputNames)), - ) - } e := b.build(a.Plan) if b.err != nil { return nil, errors.Trace(b.err) diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index d9a859b771740..51246075f3dc9 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -90,16 +90,6 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { } else { dagReq.EncodeType = tipb.EncodeType_TypeChunk } - // for _, mppTask := range pf.ExchangeSender.Tasks { - // logutil.BgLogger().Info("Dispatch mpp task", zap.Uint64("timestamp", mppTask.StartTs), - // zap.Int64("ID", mppTask.ID), zap.Uint64("QueryTs", mppTask.MppQueryID.QueryTs), zap.Uint64("LocalQueryId", mppTask.MppQueryID.LocalQueryID), - // zap.Uint64("ServerID", mppTask.MppQueryID.ServerID), zap.String("address", mppTask.Meta.GetAddress()), - // zap.String("plan", plannercore.ToStringNewForMPP(pf.ExchangeSender)), - // zap.Int64("mpp-version", mppTask.MppVersion.ToInt64()), - // zap.String("exchange-compression-mode", pf.ExchangeSender.CompressionMode.Name()), - // ) - // } - // return nil for _, mppTask := range pf.ExchangeSender.Tasks { if mppTask.PartitionTableIDs != nil { err = updateExecutorTableID(context.Background(), dagReq.RootExecutor, true, mppTask.PartitionTableIDs) @@ -242,7 +232,6 @@ func (e *MPPGather) Open(ctx context.Context) (err error) { return errors.Trace(err) } } - // return errors.Errorf("break for debugging") failpoint.Inject("checkTotalMPPTasks", func(val failpoint.Value) { if val.(int) != len(e.mppReqs) { failpoint.Return(errors.Errorf("The number of tasks is not right, expect %d tasks but actually there are %d tasks", val.(int), len(e.mppReqs))) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 58a4d22eadb42..56026e7d6f062 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2809,15 +2809,7 @@ func (p *baseLogicalPlan) canPushToCopImpl(storeTp kv.StoreType, considerDual bo case *LogicalLimit, *LogicalTopN: return false case *LogicalSequence: - if storeTp == kv.TiFlash { - for _, cte := range c.ctes { - if cte.cteClass.recursivePartLogicalPlan != nil || !cte.cteClass.seedPartLogicalPlan.canPushToCop(storeTp) { - return false - } - } - return true - } - return false + return storeTp == kv.TiFlash case *LogicalCTE: if storeTp != kv.TiFlash { return false diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 0b2e18981ded7..e4262d2be867e 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -248,9 +248,6 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } } - if _, ok := pp.(*PhysicalCTEStorage); ok { - logutil.BgLogger().Warn("xxxxx") - } // Combine best child tasks with parent physical plan. curTask := pp.attach2Task(childTasks...) diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 1cd86fa551c21..b652239ce504c 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -229,9 +229,6 @@ func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, fo cur := stack[len(stack)-1] switch x := cur.(type) { case *PhysicalTableScan, *PhysicalExchangeReceiver, *PhysicalCTE: // This should be the leave node. - if x.ID() == 359 { - logutil.BgLogger().Warn("1") - } p, err := stack[0].Clone() if err != nil { return errors.Trace(err) @@ -325,8 +322,6 @@ func (e *mppTaskGenerator) generateMPPTasksForExchangeSender(s *PhysicalExchange if cached, ok := e.cache[s.ID()]; ok { return cached.tasks, cached.frags, nil } - mppVersion := e.ctx.GetSessionVars().ChooseMppVersion() - logutil.BgLogger().Info("Mpp will generate tasks", zap.String("plan", ToStringNewForMPP(s)), zap.Int64("mpp-version", mppVersion.ToInt64())) frags, err := e.buildFragments(s) if err != nil { return nil, nil, errors.Trace(err) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 587ac1f8f9dad..ee997c66de44f 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4273,9 +4273,7 @@ func (b *PlanBuilder) tryToBuildSequence(ctes []*cteInfo, p LogicalPlan) Logical lcte.SetSchema(getResultCTESchema(cte.seedLP.Schema(), b.ctx.GetSessionVars())) lctes = append(lctes, lcte) } - seq := LogicalSequence{ - ctes: ctes, - }.Init(b.ctx, b.getSelectOffset()) + seq := LogicalSequence{}.Init(b.ctx, b.getSelectOffset()) seq.SetChildren(append(lctes, p)...) seq.SetOutputNames(p.OutputNames().Shallow()) return seq diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 54483b7b0e29a..b81dcfd388b4b 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -2043,8 +2043,6 @@ func (p *LogicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn { type LogicalSequence struct { baseLogicalPlan - - ctes []*cteInfo } func (p *LogicalSequence) Schema() *expression.Schema { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index cdb21c7752073..636a6d3bfd4b6 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1314,11 +1314,6 @@ func getPossibleAccessPaths(ctx sessionctx.Context, tableHints *tableHintInfo, i publicPaths = append(publicPaths, tablePath) if tblInfo.TiFlashReplica == nil { - - if !ctx.GetSessionVars().InRestrictedSQL { - publicPaths = append(publicPaths, genTiFlashPath(tblInfo)) - } - ctx.GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because there aren't tiflash replicas of table `" + tblInfo.Name.O + "`.") } else if !tblInfo.TiFlashReplica.Available { ctx.GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because tiflash replicas of table `" + tblInfo.Name.O + "` not ready.") diff --git a/planner/core/stats.go b/planner/core/stats.go index 09e21f4407b47..fafa3275667f5 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -116,10 +116,6 @@ func GetStats4Test(p LogicalPlan) *property.StatsInfo { } func (p *baseLogicalPlan) recursiveDeriveStats(colGroups [][]*expression.Column) (*property.StatsInfo, error) { - if len(p.self.Schema().Columns) == 0 { - logutil.BgLogger().Warn("xxxx", zap.String("the plan", fmt.Sprintf("%v", p.ExplainID())), zap.String("tree", ToString(p.self))) - return nil, errors.Errorf("?????") - } childStats := make([]*property.StatsInfo, len(p.children)) childSchema := make([]*expression.Schema, len(p.children)) cumColGroups := p.self.ExtractColGroups(colGroups) @@ -1059,32 +1055,3 @@ func (p *LogicalSequence) DeriveStats(childStats []*property.StatsInfo, _ *expre p.stats = childStats[len(childStats)-1] return p.stats, nil } - -// func (p *LogicalCTE) recursiveDeriveStats(colGroups [][]*expression.Column) (*property.StatsInfo, error) { -// if len(p.children) == 0 { -// return p.DeriveStats(nil, p.self.Schema(), nil, colGroups) -// } -// var err error -// if len(p.cte.pushDownPredicates) > 0 { -// newCond := expression.ComposeDNFCondition(p.ctx, p.cte.pushDownPredicates...) -// sel := LogicalSelection{Conditions: []expression.Expression{newCond}}.Init(p.ctx, p.blockOffset) -// sel.SetChildren(p.children[0]) -// p.SetChildren(sel) -// } -// p.children[0], err = logicalOptimize(context.TODO(), p.cte.optFlag, p.children[0]) -// if err != nil { -// return nil, err -// } -// childStats := make([]*property.StatsInfo, len(p.children)) -// childSchema := make([]*expression.Schema, len(p.children)) -// cumColGroups := p.self.ExtractColGroups(colGroups) -// for i, child := range p.children { -// childProfile, err := child.recursiveDeriveStats(cumColGroups) -// if err != nil { -// return nil, err -// } -// childStats[i] = childProfile -// childSchema[i] = child.Schema() -// } -// return p.self.DeriveStats(childStats, p.self.Schema(), childSchema, colGroups) -// } diff --git a/planner/core/stringer.go b/planner/core/stringer.go index a383871aeef15..de045c5d0b0af 100644 --- a/planner/core/stringer.go +++ b/planner/core/stringer.go @@ -22,51 +22,6 @@ import ( "github.com/pingcap/tidb/util/plancodec" ) -func ToStringNewForMPP(p PhysicalPlan) string { - builder := &strings.Builder{} - childLen := len(p.Children()) - if childLen == 1 { - if _, ok := p.(*PhysicalExchangeReceiver); !ok { - fmt.Fprintf(builder, ToStringNewForMPP(p.Children()[0])) - fmt.Fprintf(builder, "->") - } - } else if childLen > 1 { - fmt.Fprintf(builder, "(") - for i := 0; i < childLen; i++ { - fmt.Fprintf(builder, ToStringNewForMPP(p.Children()[i])) - if i != childLen-1 { - fmt.Fprintf(builder, ",") - } - } - fmt.Fprintf(builder, ")->") - } - fmt.Fprintf(builder, p.TP()) - switch x := p.(type) { - case *PhysicalExchangeSender: - fmt.Fprintf(builder, "_%d", x.id) - fmt.Fprintf(builder, "(") - for _, task := range x.TargetTasks { - fmt.Fprintf(builder, "%d, ", task.ID) - } - for _, tasks := range x.TargetCTEReaderTasks { - fmt.Fprintf(builder, "(") - for _, task := range tasks { - fmt.Fprintf(builder, "%d, ", task.ID) - } - fmt.Fprintf(builder, ")") - } - fmt.Fprintf(builder, ")") - case *PhysicalExchangeReceiver: - fmt.Fprintf(builder, "_%d", x.id) - fmt.Fprintf(builder, "(") - for _, task := range x.Tasks { - fmt.Fprintf(builder, "%d, ", task.ID) - } - fmt.Fprintf(builder, ")") - } - return builder.String() -} - // ToString explains a Plan, returns description string. func ToString(p Plan) string { strs, _ := toString(p, []string{}, []int{}) From e4010a823bce6627a2533df1df8716f1988e6253 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 4 Apr 2023 18:48:39 +0800 Subject: [PATCH 09/24] push sequence down --- planner/core/logical_plan_builder.go | 1 + planner/core/optimizer.go | 2 ++ 2 files changed, 3 insertions(+) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 8b9f03f229bd9..4fece6c448979 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4276,6 +4276,7 @@ func (b *PlanBuilder) tryToBuildSequence(ctes []*cteInfo, p LogicalPlan) Logical lcte.SetSchema(getResultCTESchema(cte.seedLP.Schema(), b.ctx.GetSessionVars())) lctes = append(lctes, lcte) } + b.optFlag |= flagPushDownSequence seq := LogicalSequence{}.Init(b.ctx, b.getSelectOffset()) seq.SetChildren(append(lctes, p)...) seq.SetOutputNames(p.OutputNames().Shallow()) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 94db123a2caae..abbc6a2e18237 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -81,6 +81,7 @@ const ( flagSyncWaitStatsLoadPoint flagJoinReOrder flagPrunColumnsAgain + flagPushDownSequence ) var optRuleList = []logicalOptRule{ @@ -105,6 +106,7 @@ var optRuleList = []logicalOptRule{ &syncWaitStatsLoadPoint{}, &joinReOrderSolver{}, &columnPruner{}, // column pruning again at last, note it will mess up the results of buildKeySolver + &pushDownSequenceSolver{}, } type logicalOptimizeOp struct { From 521d66d13da89859a5d935dd6e6c94aac772af1f Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 18 Apr 2023 22:45:40 +0800 Subject: [PATCH 10/24] Apply suggestions from code review Co-authored-by: Arenatlx --- planner/core/exhaust_physical_plans.go | 2 +- planner/core/fragment.go | 1 + store/copr/mpp.go | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 9edea6b7d44dc..706d181890b6d 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -3488,7 +3488,7 @@ func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) } possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType.CloneEssentialFields(), childProp}) } else { - possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{{TaskTp: property.RootTaskType, ExpectedCnt: math.MaxInt64}, childProp}) + possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{{TaskTp: property.RootTaskType, ExpectedCnt: math.MaxFloat64}, childProp}) } if prop.TaskTp != property.MppTaskType && prop.CTEConsumerStatus != property.SomeCTEFailedMpp && diff --git a/planner/core/fragment.go b/planner/core/fragment.go index b652239ce504c..bd4528ccf337e 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -271,6 +271,7 @@ func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, fo } case *PhysicalSequence: lastChildIdx := len(x.children) - 1 + // except the last child, those previous ones are all cte producer. for i := 0; i < lastChildIdx; i++ { if e.CTEGroups == nil { e.CTEGroups = make(map[int]*CTEGroupInFragment) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 0dd2470b75e40..7960167d26429 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -491,7 +491,7 @@ func (m *mppIterator) handleMPPStreamResponse(bo *Backoffer, response *mpp.MPPDa zap.Uint64("txnStartTS", req.StartTs), zap.String("storeAddr", req.Meta.GetAddress()), zap.Int64("mpp-version", m.mppVersion.ToInt64()), - zap.Int64("task id", req.ID), + zap.Int64("task-id", req.ID), zap.Error(err)) return err } From 3f2418a6a1a35975e1b28e4fdc4f2f7089980c32 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 18 Apr 2023 23:43:39 +0800 Subject: [PATCH 11/24] fix tests --- DEPS.bzl | 5 +++-- executor/BUILD.bazel | 1 + planner/core/BUILD.bazel | 1 + planner/core/fragment.go | 8 ++++---- planner/core/initialize.go | 2 ++ planner/core/logical_plan_builder.go | 3 +++ planner/core/logical_plans.go | 9 +++++++++ planner/core/physical_plans.go | 9 +++++++++ planner/core/rule_column_pruning.go | 1 + planner/core/rule_predicate_push_down.go | 2 ++ planner/core/rule_push_down_sequence.go | 1 + planner/core/stats.go | 1 + planner/property/physical_property.go | 9 +++++---- sessionctx/variable/session.go | 2 ++ sessionctx/variable/sysvar.go | 4 ++++ sessionctx/variable/tidb_vars.go | 4 ++++ util/plancodec/id.go | 2 +- 17 files changed, 53 insertions(+), 11 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 1013db3bbd2e0..4ccdff79970d4 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -3402,8 +3402,9 @@ def go_deps(): name = "com_github_pingcap_tipb", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/tipb", - sum = "h1:CeeMOq1aHPAhXrw4eYXtQRyWOFlbfqK1+3f9Iop4IfU=", - version = "v0.0.0-20230310043643-5362260ee6f7", + replace = "github.com/pingcap/tipb", + sum = "h1:dWYwAeMc0QQtWjVrL/mNx/3yTm8wcmcFHw4sHm8EXfU=", + version = "v0.0.0-20230418145421-de2cb4e699e9", ) go_repository( name = "com_github_pkg_browser", diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index a6ec5bc072bdc..2576bf04bde30 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -227,6 +227,7 @@ go_library( "@com_github_pingcap_kvproto//pkg/encryptionpb", "@com_github_pingcap_kvproto//pkg/kvrpcpb", "@com_github_pingcap_kvproto//pkg/metapb", + "@com_github_pingcap_kvproto//pkg/mpp", "@com_github_pingcap_kvproto//pkg/resource_manager", "@com_github_pingcap_kvproto//pkg/tikvpb", "@com_github_pingcap_log//:log", diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 125dc406af75d..64d537ba11199 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -62,6 +62,7 @@ go_library( "rule_partition_processor.go", "rule_predicate_push_down.go", "rule_predicate_simplification.go", + "rule_push_down_sequence.go", "rule_result_reorder.go", "rule_semi_join_rewrite.go", "rule_topn_push_down.go", diff --git a/planner/core/fragment.go b/planner/core/fragment.go index c0ba6a6027c6e..97f88c8d1e7ea 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -56,7 +56,7 @@ type Fragment struct { singleton bool // indicates if this is a task running on a single node. } -type CTEGroupInFragment struct { +type cteGroupInFragment struct { CTEStorage *PhysicalCTEStorage CTEReader []*PhysicalCTE @@ -99,7 +99,7 @@ type mppTaskGenerator struct { frags []*Fragment cache map[int]tasksAndFrags - CTEGroups map[int]*CTEGroupInFragment + CTEGroups map[int]*cteGroupInFragment } // GenerateRootMPPTasks generate all mpp tasks and return root ones. @@ -274,10 +274,10 @@ func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, fo // except the last child, those previous ones are all cte producer. for i := 0; i < lastChildIdx; i++ { if e.CTEGroups == nil { - e.CTEGroups = make(map[int]*CTEGroupInFragment) + e.CTEGroups = make(map[int]*cteGroupInFragment) } cteStorage := x.children[i].(*PhysicalCTEStorage) - e.CTEGroups[cteStorage.CTE.IDForStorage] = &CTEGroupInFragment{ + e.CTEGroups[cteStorage.CTE.IDForStorage] = &cteGroupInFragment{ CTEStorage: cteStorage, CTEReader: make([]*PhysicalCTE, 0, 3), } diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 6421b16fc4192..ea62c7cd86210 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -652,11 +652,13 @@ func (p FKCascade) Init(ctx sessionctx.Context) *FKCascade { return &p } +// Init initializes LogicalSequence func (p LogicalSequence) Init(ctx sessionctx.Context, offset int) *LogicalSequence { p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeSequence, &p, offset) return &p } +// Init initializes PhysicalSequence func (p PhysicalSequence) Init(ctx sessionctx.Context, stats *property.StatsInfo, blockOffset int, props ...*property.PhysicalProperty) *PhysicalSequence { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeSequence, &p, blockOffset) p.stats = stats diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 0f395d9800324..a1f35c9894a1a 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4253,6 +4253,9 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L } func (b *PlanBuilder) tryToBuildSequence(ctes []*cteInfo, p LogicalPlan) LogicalPlan { + if !b.ctx.GetSessionVars().EnableMPPSharedCTEExecution { + return p + } for i := len(ctes) - 1; i >= 0; i-- { if !ctes[i].nonRecursive { return p diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index b121abcfa2c2e..d01d344003a55 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -2049,10 +2049,19 @@ func (p *LogicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn { return corCols } +// LogicalSequence is used to mark the CTE producer in the main query tree. +// Its last child is main query. The previous children are cte producers. +// And there might be dependencies between the CTE producers: +// Suppose that the sequence has 4 children, naming c0, c1, c2, c3. +// From the definition, c3 is the main query. c0, c1, c2 are CTE producers. +// It's possible that c1 referneces c0, c2 references c1 and c2. +// But it's no possible that c0 references c1 or c2. +// We use this property to do complex optimizations for CTEs. type LogicalSequence struct { baseLogicalPlan } +// Schema returns its last child(which is the main query plan)'s schema. func (p *LogicalSequence) Schema() *expression.Schema { return p.children[len(p.children)-1].Schema() } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 1b0c9fb5336e0..ba3affa0666fa 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -2486,6 +2486,7 @@ func (p *PhysicalCTE) ExplainID() fmt.Stringer { }) } +// Clone implementes PhysicalPlan interface. func (p *PhysicalCTE) Clone() (PhysicalPlan, error) { cloned := new(PhysicalCTE) base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) @@ -2600,6 +2601,7 @@ func (p *CTEDefinition) MemoryUsage() (sum int64) { return } +// PhysicalCTEStorage is used for representing CTE storage, or CTE producer in other words. type PhysicalCTEStorage PhysicalCTE // ExplainInfo overrides the ExplainInfo @@ -2627,6 +2629,7 @@ func (p *PhysicalCTEStorage) MemoryUsage() (sum int64) { return } +// Clone implements PhysicalPlan interface. func (p *PhysicalCTEStorage) Clone() (PhysicalPlan, error) { cloned, err := (*PhysicalCTE)(p).Clone() if err != nil { @@ -2649,10 +2652,12 @@ func appendChildCandidate(origin PhysicalPlan, pp PhysicalPlan, op *physicalOpti op.tracer.Candidates[origin.ID()].AppendChildrenID(pp.ID()) } +// PhysicalSequence is the physical representation of LogicalSequence. Used to mark the CTE producers in the plan tree. type PhysicalSequence struct { physicalSchemaProducer } +// MemoryUsage returns the memory usage of the PhysicalSequence. func (p *PhysicalSequence) MemoryUsage() (sum int64) { if p == nil { return @@ -2663,6 +2668,7 @@ func (p *PhysicalSequence) MemoryUsage() (sum int64) { return } +// ExplainID overrides the ExplainID. func (p *PhysicalSequence) ExplainID() fmt.Stringer { return stringutil.MemoizeStr(func() string { if p.ctx != nil && p.ctx.GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { @@ -2672,12 +2678,14 @@ func (p *PhysicalSequence) ExplainID() fmt.Stringer { }) } +// ExplainInfo overrides the ExplainInfo. func (p *PhysicalSequence) ExplainInfo() string { var res string res = "Sequence Node" return res } +// Clone implements PhysicalPlan interface. func (p *PhysicalSequence) Clone() (PhysicalPlan, error) { cloned := new(PhysicalSequence) base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) @@ -2688,6 +2696,7 @@ func (p *PhysicalSequence) Clone() (PhysicalPlan, error) { return cloned, nil } +// Schema returns its last child(which is the main query tree)'s schema. func (p *PhysicalSequence) Schema() *expression.Schema { return p.Children()[len(p.Children())-1].Schema() } diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 973714c7c3c54..fded2dd991acb 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -673,6 +673,7 @@ func (p *LogicalCTE) PruneColumns(parentUsedCols []*expression.Column, opt *logi return nil } +// PruneColumns implements the interface of LogicalPlan. func (p *LogicalSequence) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { return p.children[len(p.children)-1].PruneColumns(parentUsedCols, opt) } diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 62f74cc390f4f..cce05c9da3f07 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -1001,6 +1001,8 @@ func (p *LogicalCTE) PredicatePushDown(predicates []expression.Expression, _ *lo return predicates, p.self } +// PredicatePushDown implements the LogicalPlan interface. +// Currently, we only maintain the main query tree. func (p *LogicalSequence) PredicatePushDown(predicates []expression.Expression, op *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { lastIdx := len(p.children) - 1 remained, newLastChild := p.children[lastIdx].PredicatePushDown(predicates, op) diff --git a/planner/core/rule_push_down_sequence.go b/planner/core/rule_push_down_sequence.go index b04bd58058c30..2b29ab0c3a281 100644 --- a/planner/core/rule_push_down_sequence.go +++ b/planner/core/rule_push_down_sequence.go @@ -11,6 +11,7 @@ // 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 core import "context" diff --git a/planner/core/stats.go b/planner/core/stats.go index 03ef5ff6cc1f4..7b38de6e1316a 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -1094,6 +1094,7 @@ func (p *LogicalCTETable) DeriveStats(_ []*property.StatsInfo, _ *expression.Sch return p.stats, nil } +// DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalSequence) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { p.stats = childStats[len(childStats)-1] return p.stats, nil diff --git a/planner/property/physical_property.go b/planner/property/physical_property.go index 0f62ff22b7813..6d4d3c6136e92 100644 --- a/planner/property/physical_property.go +++ b/planner/property/physical_property.go @@ -161,11 +161,12 @@ func GetCollateNameByIDForPartition(collateID int32) string { return collate.CollationID2Name(collateID) } -// cteConsumerStatus indicates whether we can let the current CTE consumer/reader be executed on the MPP nodes. -type cteConsumerStatus int +// cteProducerStatus indicates whether we can let the current CTE consumer/reader be executed on the MPP nodes. +type cteProducerStatus int +// Constants for CTE status. const ( - NoCTE cteConsumerStatus = iota + NoCTE cteProducerStatus = iota SomeCTEFailedMpp AllCTECanMpp ) @@ -212,7 +213,7 @@ type PhysicalProperty struct { // Non-MPP tasks do not care about it. RejectSort bool - CTEConsumerStatus cteConsumerStatus + CTEConsumerStatus cteProducerStatus } // NewPhysicalProperty builds property from columns. diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 3937e90e7b7c9..a7640636c8ec9 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1423,6 +1423,8 @@ type SessionVars struct { // If there exists an index whose estimated selectivity is smaller than this threshold, the optimizer won't // use the ExpectedCnt to adjust the estimated row count for index scan. OptOrderingIdxSelThresh float64 + + EnableMPPSharedCTEExecution bool } // planReplayerSessionFinishedTaskKeyLen is used to control the max size for the finished plan replayer task key in session diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index b08e24b5954ce..0f95c6f7cff89 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -2461,6 +2461,10 @@ var defaultSysVars = []*SysVar{ s.OptOrderingIdxSelThresh = tidbOptFloat64(val, DefTiDBOptOrderingIdxSelThresh) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptEnableMPPSharedCTEExecution, Value: BoolToOnOff(DefTiDBOptEnableMPPSharedCTEExecution), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.EnableMPPSharedCTEExecution = TiDBOptOn(val) + return nil + }}, } func setTiFlashComputeDispatchPolicy(s *SessionVars, val string) error { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 20a5ae34e1b80..ca66ae3a5f35d 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -855,6 +855,9 @@ const ( // TiDBOptOrderingIdxSelThresh is the threshold for optimizer to consider the ordering index. TiDBOptOrderingIdxSelThresh = "tidb_opt_ordering_index_selectivity_threshold" + + // TiDBOptEnableMPPSharedCTEExecution indicates whehter the optimizer try to build shared CTE scan during MPP execution. + TiDBOptEnableMPPSharedCTEExecution = "tidb_opt_enable_mpp_shared_cte_execution" ) // TiDB vars that have only global scope @@ -1256,6 +1259,7 @@ const ( DefTiDBLoadBasedReplicaReadThreshold = 0 DefTiDBOptEnableLateMaterialization = true DefTiDBOptOrderingIdxSelThresh = 0.0 + DefTiDBOptEnableMPPSharedCTEExecution = false ) // Process global variables. diff --git a/util/plancodec/id.go b/util/plancodec/id.go index 92c3975dc8a03..c23959072e4f6 100644 --- a/util/plancodec/id.go +++ b/util/plancodec/id.go @@ -133,7 +133,7 @@ const ( TypeForeignKeyCheck = "Foreign_Key_Check" // TypeForeignKeyCascade is the type of FKCascade TypeForeignKeyCascade = "Foreign_Key_Cascade" - // TypeSequence + // TypeSequence is the type of Sequence TypeSequence = "Sequence" ) From 671ef17884763c8516ea923d4f6118c610879ceb Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 24 Apr 2023 00:19:10 +0800 Subject: [PATCH 12/24] address comments && add tests --- planner/core/casetest/enforce_mpp_test.go | 48 ++++++++ .../testdata/enforce_mpp_suite_in.json | 13 +++ .../testdata/enforce_mpp_suite_out.json | 107 ++++++++++++++++++ planner/core/exhaust_physical_plans.go | 38 +++---- planner/core/find_best_task.go | 8 +- planner/core/fragment.go | 18 +-- planner/core/physical_plans.go | 11 +- planner/property/physical_property.go | 6 +- 8 files changed, 208 insertions(+), 41 deletions(-) diff --git a/planner/core/casetest/enforce_mpp_test.go b/planner/core/casetest/enforce_mpp_test.go index 7f6fb0b990e2d..09499a74de87c 100644 --- a/planner/core/casetest/enforce_mpp_test.go +++ b/planner/core/casetest/enforce_mpp_test.go @@ -598,3 +598,51 @@ func TestMPPNullAwareSemiJoinPushDown(t *testing.T) { require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } + +func TestMPPSharedCTEScan(t *testing.T) { + store := testkit.CreateMockStore(t, internal.WithMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists s") + tk.MustExec("create table t(a int, b int, c int)") + tk.MustExec("create table s(a int, b int, c int)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("alter table s set tiflash replica 1") + + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tb = external.GetTableByName(t, tk, "test", "s") + err = domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + + tk.MustExec("set @@tidb_enforce_mpp='on'") + tk.MustExec("set @@tidb_opt_enable_mpp_shared_cte_execution='on'") + + enforceMPPSuiteData := GetEnforceMPPSuiteData() + enforceMPPSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} diff --git a/planner/core/casetest/testdata/enforce_mpp_suite_in.json b/planner/core/casetest/testdata/enforce_mpp_suite_in.json index 33f4893d96a56..e2b1ad3224b75 100644 --- a/planner/core/casetest/testdata/enforce_mpp_suite_in.json +++ b/planner/core/casetest/testdata/enforce_mpp_suite_in.json @@ -172,5 +172,18 @@ "EXPLAIN select *, (t.a, t.b) not in (select s.a, s.b from s) from t; -- 7. left anti semi join, two join key", "EXPLAIN select *, (t.a, t.b) not in (select s.a, s.b from s where t.c < s.c) from t; -- 8. left anti semi join, two join key + other condition" ] + }, + { + "name": "TestMPPSharedCTEScan", + "cases": [ + // The most simple case. + "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b ", + "explain with c1 as (select * from t) select c1.* from c1, c1 c2 where c1.b=c2.c", + // Can work when there's global limit/topn + "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b limit 10", + "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b order by c1.a limit 10", + // The c2 references c1, c1 can mpp, and then c2 can mpp, so the main query can mpp. + "explain with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b" + ] } ] diff --git a/planner/core/casetest/testdata/enforce_mpp_suite_out.json b/planner/core/casetest/testdata/enforce_mpp_suite_out.json index 172d6e5802225..adf4855e7ff4c 100644 --- a/planner/core/casetest/testdata/enforce_mpp_suite_out.json +++ b/planner/core/casetest/testdata/enforce_mpp_suite_out.json @@ -1588,5 +1588,112 @@ "Warn": null } ] + }, + { + "Name": "TestMPPSharedCTEScan", + "Cases": [ + { + "SQL": "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b ", + "Plan": [ + "TableReader_70 8000.00 root MppVersion: 1, data:ExchangeSender_69", + "└─ExchangeSender_69 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Sequence_68 8000.00 mpp[tiflash] Sequence Node", + " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection_53 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan_21 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin_54 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver_62(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender_61 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_56 6400.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan_59 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection_63(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan_64 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + ], + "Warn": null + }, + { + "SQL": "explain with c1 as (select * from t) select c1.* from c1, c1 c2 where c1.b=c2.c", + "Plan": [ + "TableReader_70 8000.00 root MppVersion: 1, data:ExchangeSender_69", + "└─ExchangeSender_69 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_30 8000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " └─Sequence_68 8000.00 mpp[tiflash] Sequence Node", + " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection_53 8000.00 mpp[tiflash] or(not(isnull(test.t.b)), not(isnull(test.t.c)))", + " │ └─TableFullScan_21 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin_54 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─ExchangeReceiver_62(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender_61 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_56 6400.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─CTEFullScan_59 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection_63(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.c))", + " └─CTEFullScan_64 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + ], + "Warn": null + }, + { + "SQL": "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b limit 10", + "Plan": [ + "Limit_36 10.00 root offset:0, count:10", + "└─TableReader_89 10.00 root MppVersion: 1, data:ExchangeSender_88", + " └─ExchangeSender_88 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit_87 10.00 mpp[tiflash] offset:0, count:10", + " └─Sequence_82 8000.00 mpp[tiflash] Sequence Node", + " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection_67 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan_24 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin_83 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver_76(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender_75 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_70 6400.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan_73 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection_84(Probe) 8.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan_85 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + ], + "Warn": null + }, + { + "SQL": "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b order by c1.a limit 10", + "Plan": [ + "TopN_36 10.00 root test.t.a, offset:0, count:10", + "└─TableReader_80 10.00 root MppVersion: 1, data:ExchangeSender_79", + " └─ExchangeSender_79 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN_78 10.00 mpp[tiflash] test.t.a, offset:0, count:10", + " └─Sequence_77 8000.00 mpp[tiflash] Sequence Node", + " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection_62 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan_24 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin_63 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver_71(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender_70 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_65 6400.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan_68 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection_72(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan_73 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + ], + "Warn": null + }, + { + "SQL": "explain with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b", + "Plan": [ + "HashJoin_60 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", + "├─Selection_72(Build) 8000.00 root not(isnull(test.t.b))", + "│ └─CTEFullScan_74 10000.00 root CTE:c2 data:CTE_1", + "└─Selection_68(Probe) 8000.00 root not(isnull(test.t.a))", + " └─CTEFullScan_70 10000.00 root CTE:c2 AS c1 data:CTE_1", + "CTE_1 10000.00 root Non-Recursive CTE", + "└─HashJoin_34(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─Selection_46(Build) 8000.00 root not(isnull(test.t.c))", + " │ └─CTEFullScan_48 10000.00 root CTE:c1 AS c2 data:CTE_0", + " └─Selection_42(Probe) 8000.00 root not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─CTEFullScan_44 10000.00 root CTE:c1 data:CTE_0", + "CTE_0 10000.00 root Non-Recursive CTE", + "└─TableReader_29(Seed Part) 10000.00 root MppVersion: 1, data:ExchangeSender_28", + " └─ExchangeSender_28 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + } + ] } ] diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 5c8294cd3387b..ebdd13591ac86 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2424,7 +2424,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC baseJoin.InnerChildIdx = preferredBuildIndex childrenProps := make([]*property.PhysicalProperty, 2) if useBCJ { - childrenProps[preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.BroadcastType, CanAddEnforcer: true, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} + childrenProps[preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.BroadcastType, CanAddEnforcer: true, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} expCnt := math.MaxFloat64 if prop.ExpectedCnt < p.stats.RowCount { expCntScale := prop.ExpectedCnt / p.stats.RowCount @@ -2437,12 +2437,12 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC hashKeys = lPartitionKeys } if matches := prop.IsSubsetOf(hashKeys); len(matches) != 0 { - childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.HashType, MPPPartitionCols: prop.MPPPartitionCols, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} + childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.HashType, MPPPartitionCols: prop.MPPPartitionCols, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} } else { return nil } } else { - childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.AnyType, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} + childrenProps[1-preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: expCnt, MPPPartitionTp: property.AnyType, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} } } else { lPartitionKeys, rPartitionKeys := p.GetPotentialPartitionKeys() @@ -2471,8 +2471,8 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC lPartitionKeys = choosePartitionKeys(lPartitionKeys, matches) rPartitionKeys = choosePartitionKeys(rPartitionKeys, matches) } - childrenProps[0] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: lPartitionKeys, CanAddEnforcer: true, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} - childrenProps[1] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: rPartitionKeys, CanAddEnforcer: true, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} + childrenProps[0] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: lPartitionKeys, CanAddEnforcer: true, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} + childrenProps[1] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: rPartitionKeys, CanAddEnforcer: true, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} } join := PhysicalHashJoin{ basePhysicalJoin: baseJoin, @@ -2586,7 +2586,7 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { - resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: math.MaxFloat64, CTEConsumerStatus: prop.CTEConsumerStatus} + resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus} topN := PhysicalTopN{ ByItems: lt.ByItems, PartitionBy: lt.PartitionBy, @@ -2763,7 +2763,7 @@ func (lw *LogicalWindow) tryToGetMppWindows(prop *property.PhysicalProperty) []P SortItems: byItems, TaskTp: property.MppTaskType, SortItemsForPartition: byItems, - CTEConsumerStatus: prop.CTEConsumerStatus, + CTEProducerStatus: prop.CTEProducerStatus, } if !prop.IsPrefix(childProperty) { return nil @@ -3112,7 +3112,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // To avoid mess, we don't do any one-phase aggregation in this case. // If this is a skew distinct group agg, skip generating 1-phase agg, because skew data will cause performance issue if len(partitionCols) != 0 && !la.ctx.GetSessionVars().EnableSkewDistinctAgg { - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: partitionCols, CanAddEnforcer: true, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: partitionCols, CanAddEnforcer: true, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = Mpp1Phase @@ -3126,7 +3126,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // 2-phase agg // no partition property down,record partition cols inside agg itself, enforce shuffler latter. - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = Mpp2Phase @@ -3135,7 +3135,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // agg runs on TiDB with a partial agg on TiFlash if possible if prop.TaskTp == property.RootTaskType { - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = MppTiDB @@ -3143,7 +3143,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert } } else if !hasFinalAgg { // TODO: support scalar agg in MPP, merge the final result to one node - childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} + childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) if la.HasDistinct() || la.HasOrderBy() { @@ -3341,7 +3341,7 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { - resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(p.Count + p.Offset), CTEConsumerStatus: prop.CTEConsumerStatus} + resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(p.Count + p.Offset), CTEProducerStatus: prop.CTEProducerStatus} limit := PhysicalLimit{ Offset: p.Offset, Count: p.Count, @@ -3385,7 +3385,7 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ExpectedCnt: prop.ExpectedCnt, TaskTp: property.MppTaskType, RejectSort: true, - CTEConsumerStatus: prop.CTEConsumerStatus, + CTEProducerStatus: prop.CTEProducerStatus, }) } else { chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: prop.ExpectedCnt, RejectSort: true}) @@ -3423,7 +3423,7 @@ func (p *LogicalPartitionUnionAll) exhaustPhysicalPlans(prop *property.PhysicalP } func (ls *LogicalSort) getPhysicalSort(prop *property.PhysicalProperty) *PhysicalSort { - ps := PhysicalSort{ByItems: ls.ByItems}.Init(ls.ctx, ls.stats.ScaleByExpectCnt(prop.ExpectedCnt), ls.blockOffset, &property.PhysicalProperty{TaskTp: prop.TaskTp, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus}) + ps := PhysicalSort{ByItems: ls.ByItems}.Init(ls.ctx, ls.stats.ScaleByExpectCnt(prop.ExpectedCnt), ls.blockOffset, &property.PhysicalProperty{TaskTp: prop.TaskTp, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus}) return ps } @@ -3486,19 +3486,19 @@ func (p *LogicalCTE) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]Ph func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { possibleChildrenProps := make([][]*property.PhysicalProperty, 0, 2) childProp := prop.CloneEssentialFields() - anyType := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, CanAddEnforcer: true, RejectSort: true, CTEConsumerStatus: prop.CTEConsumerStatus} + anyType := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, CanAddEnforcer: true, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} if prop.TaskTp == property.MppTaskType { - if prop.CTEConsumerStatus == property.SomeCTEFailedMpp { + if prop.CTEProducerStatus == property.SomeCTEFailedMpp { return nil, true, nil } - possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType.CloneEssentialFields(), childProp}) + possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType, childProp}) } else { possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{{TaskTp: property.RootTaskType, ExpectedCnt: math.MaxFloat64}, childProp}) } - if prop.TaskTp != property.MppTaskType && prop.CTEConsumerStatus != property.SomeCTEFailedMpp && + if prop.TaskTp != property.MppTaskType && prop.CTEProducerStatus != property.SomeCTEFailedMpp && p.SCtx().GetSessionVars().IsMPPAllowed() && prop.IsSortItemEmpty() { - possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType.CloneEssentialFields(), anyType.CloneEssentialFields()}) + possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType, anyType.CloneEssentialFields()}) } seqs := make([]PhysicalPlan, 0, 2) for _, propChoice := range possibleChildrenProps { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index f8d7460d205c0..c8d00300e2e51 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -329,7 +329,7 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl lastChildProp := pp.GetChildReqProps(lastIdx).CloneEssentialFields() if lastChildProp.IsFlashProp() { - lastChildProp.CTEConsumerStatus = property.AllCTECanMpp + lastChildProp.CTEProducerStatus = property.AllCTECanMpp } lastChildTask, cnt, err := p.Children()[lastIdx].findBestTask(lastChildProp, &PlanCounterDisabled, opt) childCnts[lastIdx] = cnt @@ -341,7 +341,7 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl continue } - if _, ok := lastChildTask.(*mppTask); !ok && lastChildProp.CTEConsumerStatus == property.AllCTECanMpp { + if _, ok := lastChildTask.(*mppTask); !ok && lastChildProp.CTEProducerStatus == property.AllCTECanMpp { continue } @@ -2670,8 +2670,8 @@ func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *Plan // The physical plan has been build when derive stats. pcte := PhysicalCTE{SeedPlan: p.cte.seedPartPhysicalPlan, RecurPlan: p.cte.recursivePartPhysicalPlan, CTE: p.cte, cteAsName: p.cteAsName, cteName: p.cteName}.Init(p.ctx, p.stats) pcte.SetSchema(p.schema) - if prop.IsFlashProp() && prop.CTEConsumerStatus == property.AllCTECanMpp { - pcte.readerRecevier = PhysicalExchangeReceiver{IsCTEReader: true}.Init(p.ctx, p.stats) + if prop.IsFlashProp() && prop.CTEProducerStatus == property.AllCTECanMpp { + pcte.readerReceiver = PhysicalExchangeReceiver{IsCTEReader: true}.Init(p.ctx, p.stats) if prop.MPPPartitionTp != property.AnyType { return invalidTask, 1, nil } diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 97f88c8d1e7ea..9cca08cc017c2 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -60,8 +60,8 @@ type cteGroupInFragment struct { CTEStorage *PhysicalCTEStorage CTEReader []*PhysicalCTE - StroageTasks []*kv.MPPTask - StroageFragments []*Fragment + StorageTasks []*kv.MPPTask + StorageFragments []*Fragment } const emptyFragmentSize = int64(unsafe.Sizeof(Fragment{})) @@ -271,7 +271,7 @@ func (e *mppTaskGenerator) untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, fo } case *PhysicalSequence: lastChildIdx := len(x.children) - 1 - // except the last child, those previous ones are all cte producer. + // except the last child, those previous ones are all cte producer. for i := 0; i < lastChildIdx; i++ { if e.CTEGroups == nil { e.CTEGroups = make(map[int]*cteGroupInFragment) @@ -412,16 +412,16 @@ func (f *Fragment) flipCTEReader(currentPlan PhysicalPlan) { func (e *mppTaskGenerator) generateTasksForCTEReader(cteReader *PhysicalCTE) (err error) { group := e.CTEGroups[cteReader.CTE.IDForStorage] - if group.StroageFragments == nil { + if group.StorageFragments == nil { group.CTEStorage.storageSender.SetChildren(group.CTEStorage.children...) - group.StroageTasks, group.StroageFragments, err = e.generateMPPTasksForExchangeSender(group.CTEStorage.storageSender) + group.StorageTasks, group.StorageFragments, err = e.generateMPPTasksForExchangeSender(group.CTEStorage.storageSender) if err != nil { return err } } - receiver := cteReader.readerRecevier - receiver.Tasks = group.StroageTasks - receiver.frags = group.StroageFragments + receiver := cteReader.readerReceiver + receiver.Tasks = group.StorageTasks + receiver.frags = group.StorageFragments cteReader.SetChildren(receiver) receiver.SetChildren(group.CTEStorage.children[0]) inconsistenceNullable := false @@ -446,7 +446,7 @@ func (e *mppTaskGenerator) generateTasksForCTEReader(cteReader *PhysicalCTE) (er func (e *mppTaskGenerator) addReaderTasksForCTEStorage(storageID int, tasks ...*kv.MPPTask) { group := e.CTEGroups[storageID] - for _, frag := range group.StroageFragments { + for _, frag := range group.StorageFragments { frag.ExchangeSender.TargetCTEReaderTasks = append(frag.ExchangeSender.TargetCTEReaderTasks, tasks) } } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index ba3affa0666fa..c3ab747c31d24 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -2446,7 +2446,7 @@ type PhysicalCTE struct { cteAsName model.CIStr cteName model.CIStr - readerRecevier *PhysicalExchangeReceiver + readerReceiver *PhysicalExchangeReceiver storageSender *PhysicalExchangeSender } @@ -2515,12 +2515,12 @@ func (p *PhysicalCTE) Clone() (PhysicalPlan, error) { } cloned.storageSender = clonedSender.(*PhysicalExchangeSender) } - if p.readerRecevier != nil { - clonedReceiver, err := p.readerRecevier.Clone() + if p.readerReceiver != nil { + clonedReceiver, err := p.readerReceiver.Clone() if err != nil { return nil, err } - cloned.readerRecevier = clonedReceiver.(*PhysicalExchangeReceiver) + cloned.readerReceiver = clonedReceiver.(*PhysicalExchangeReceiver) } return cloned, nil } @@ -2680,8 +2680,7 @@ func (p *PhysicalSequence) ExplainID() fmt.Stringer { // ExplainInfo overrides the ExplainInfo. func (p *PhysicalSequence) ExplainInfo() string { - var res string - res = "Sequence Node" + res := "Sequence Node" return res } diff --git a/planner/property/physical_property.go b/planner/property/physical_property.go index 6d4d3c6136e92..e5a289f2302c4 100644 --- a/planner/property/physical_property.go +++ b/planner/property/physical_property.go @@ -213,7 +213,7 @@ type PhysicalProperty struct { // Non-MPP tasks do not care about it. RejectSort bool - CTEConsumerStatus cteProducerStatus + CTEProducerStatus cteProducerStatus } // NewPhysicalProperty builds property from columns. @@ -344,7 +344,7 @@ func (p *PhysicalProperty) HashCode() []byte { p.hashcode = append(p.hashcode, col.hashCode(nil)...) } } - p.hashcode = append(p.hashcode, codec.EncodeInt(nil, int64(p.CTEConsumerStatus))...) + p.hashcode = append(p.hashcode, codec.EncodeInt(nil, int64(p.CTEProducerStatus))...) return p.hashcode } @@ -364,7 +364,7 @@ func (p *PhysicalProperty) CloneEssentialFields() *PhysicalProperty { MPPPartitionTp: p.MPPPartitionTp, MPPPartitionCols: p.MPPPartitionCols, RejectSort: p.RejectSort, - CTEConsumerStatus: p.CTEConsumerStatus, + CTEProducerStatus: p.CTEProducerStatus, } return prop } From 7e918fccf90f88ed0dea8147b593b565066b1875 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 24 Apr 2023 02:14:22 +0800 Subject: [PATCH 13/24] fix gofmt --- planner/core/flat_plan.go | 2 +- planner/core/logical_plans.go | 10 ++++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/planner/core/flat_plan.go b/planner/core/flat_plan.go index 5efef86a84f43..ff2bc1d7985fe 100644 --- a/planner/core/flat_plan.go +++ b/planner/core/flat_plan.go @@ -357,7 +357,7 @@ func (f *FlatPhysicalPlan) flattenRecursively(p Plan, info *operatorCtx, target copiedCTE := *plan copiedCTE.probeParents = nil if info.isRoot { - // If it's executed in TiDB, we need to record it since we don't have producer and consumer + // If it's executed in TiDB, we need to record it since we don't have producer and consumer f.ctesToFlatten = append(f.ctesToFlatten, &copiedCTE) } case *Insert: diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 08d5c2fa413ec..6edcda43e5e9d 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -2065,10 +2065,12 @@ func (p *LogicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn { // LogicalSequence is used to mark the CTE producer in the main query tree. // Its last child is main query. The previous children are cte producers. // And there might be dependencies between the CTE producers: -// Suppose that the sequence has 4 children, naming c0, c1, c2, c3. -// From the definition, c3 is the main query. c0, c1, c2 are CTE producers. -// It's possible that c1 referneces c0, c2 references c1 and c2. -// But it's no possible that c0 references c1 or c2. +// +// Suppose that the sequence has 4 children, naming c0, c1, c2, c3. +// From the definition, c3 is the main query. c0, c1, c2 are CTE producers. +// It's possible that c1 referneces c0, c2 references c1 and c2. +// But it's no possible that c0 references c1 or c2. +// // We use this property to do complex optimizations for CTEs. type LogicalSequence struct { baseLogicalPlan From 2341c885d4bab203cc1d97f10d53427634fa9761 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 24 Apr 2023 02:45:04 +0800 Subject: [PATCH 14/24] fix check && fix test --- cmd/explaintest/r/explain_cte.result | 2 +- executor/mpp_gather.go | 5 ++- .../testdata/binary_plan_suite_out.json | 2 ++ planner/core/find_best_task.go | 5 ++- planner/core/fragment.go | 5 ++- planner/core/logical_plans.go | 2 +- planner/core/optimizer.go | 31 +++++++++++-------- planner/core/physical_plans.go | 6 ++-- planner/core/rule_column_pruning.go | 2 +- planner/core/rule_push_down_sequence.go | 2 +- planner/core/task.go | 1 - 11 files changed, 37 insertions(+), 26 deletions(-) diff --git a/cmd/explaintest/r/explain_cte.result b/cmd/explaintest/r/explain_cte.result index 7d8b328fb4c50..84c8f247a5995 100644 --- a/cmd/explaintest/r/explain_cte.result +++ b/cmd/explaintest/r/explain_cte.result @@ -194,7 +194,7 @@ CTE_0 100.00 root Non-Recursive CTE └─IndexFullScan_31 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo explain with recursive cte1(c1) as (select c1 from t1 union select c1 from t2 limit 0 offset 0) select * from cte1; id estRows task access object operator info -CTEFullScan_18 0.00 root CTE:cte1 data:CTE_0 +CTEFullScan_19 0.00 root CTE:cte1 data:CTE_0 CTE_0 0.00 root Non-Recursive CTE └─TableDual_16(Seed Part) 0.00 root rows:0 CREATE TABLE `customer` ( diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 51246075f3dc9..075275caa4dc2 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -203,7 +203,10 @@ func (e *MPPGather) fixTaskForCTEStorageAndReader(exec *tipb.Executor, meta kv.M return errors.Errorf("unknown new tipb protocol %d", exec.Tp) } for _, child := range children { - e.fixTaskForCTEStorageAndReader(child, meta) + err := e.fixTaskForCTEStorageAndReader(child, meta) + if err != nil { + return err + } } return nil } diff --git a/planner/core/casetest/testdata/binary_plan_suite_out.json b/planner/core/casetest/testdata/binary_plan_suite_out.json index a361c5ca0b23f..0ce7f545d22cf 100644 --- a/planner/core/casetest/testdata/binary_plan_suite_out.json +++ b/planner/core/casetest/testdata/binary_plan_suite_out.json @@ -297,6 +297,7 @@ "BinaryPlan": { "main": { "name": "CTEFullScan_17", + "cost": 0.8982000000000001, "est_rows": 1.8, "act_rows": 5, "task_type": 1, @@ -363,6 +364,7 @@ "operator_info": "cast(plus(Column#3, 1), bigint(1) BINARY)->Column#5" } ], + "cost": 0.8982000000000001, "est_rows": 1.8, "act_rows": 5, "task_type": 1, diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 56dfcca26c5fe..0a9842de85830 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -302,7 +302,7 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl // The curCntPlan records the number of possible plans for pp curCntPlan = 1 timeStampNow := p.GetLogicalTS4TaskMap() - savedPlanID := p.ctx.GetSessionVars().PlanID + savedPlanID := p.ctx.GetSessionVars().PlanID.Load() lastIdx := len(p.children) - 1 for j := 0; j < lastIdx; j++ { child := p.children[j] @@ -349,7 +349,7 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl // If the target plan can be found in this physicalPlan(pp), rebuild childTasks to build the corresponding combination. if planCounter.IsForce() && int64(*planCounter) <= curCntPlan { - p.ctx.GetSessionVars().PlanID = savedPlanID + p.ctx.GetSessionVars().PlanID.Store(savedPlanID) curCntPlan = int64(*planCounter) err := p.rebuildChildTasks(&childTasks, pp, childCnts, int64(*planCounter), timeStampNow, opt) if err != nil { @@ -610,7 +610,6 @@ func (p *LogicalSequence) findBestTask(prop *property.PhysicalProperty, planCoun return invalidTask, 0, nil } - bestTask = invalidTask cntPlan = 0 // prop should be read only because its cached hashcode might be not consistent // when it is changed. So we clone a new one for the temporary changes. diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 9cca08cc017c2..5c7705006762a 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -343,7 +343,10 @@ func (e *mppTaskGenerator) generateMPPTasksForExchangeSender(s *PhysicalExchange func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv.MPPTask, err error) { for _, cteReader := range f.CTEReaders { - e.generateTasksForCTEReader(cteReader) + err := e.generateTasksForCTEReader(cteReader) + if err != nil { + return nil, err + } } for _, r := range f.ExchangeReceivers { // chain call: to get lower fragments and tasks diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 6edcda43e5e9d..82488890777bc 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -2068,7 +2068,7 @@ func (p *LogicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn { // // Suppose that the sequence has 4 children, naming c0, c1, c2, c3. // From the definition, c3 is the main query. c0, c1, c2 are CTE producers. -// It's possible that c1 referneces c0, c2 references c1 and c2. +// It's possible that c1 references c0, c2 references c1 and c2. // But it's no possible that c0 references c1 or c2. // // We use this property to do complex optimizations for CTEs. diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index aea24a18394c1..43a75d1000b1a 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -282,6 +282,19 @@ func checkStableResultMode(sctx sessionctx.Context) bool { // DoOptimizeAndLogicAsRet optimizes a logical plan to a physical plan and return the optimized logical plan. func DoOptimizeAndLogicAsRet(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (LogicalPlan, PhysicalPlan, float64, error) { sessVars := sctx.GetSessionVars() + // if there is something after flagPrunColumns, do flagPrunColumnsAgain + if flag&flagPrunColumns > 0 && flag-flagPrunColumns > flagPrunColumns { + flag |= flagPrunColumnsAgain + } + if checkStableResultMode(logic.SCtx()) { + flag |= flagStabilizeResults + } + if logic.SCtx().GetSessionVars().StmtCtx.StraightJoinOrder { + // When we use the straight Join Order hint, we should disable the join reorder optimization. + flag &= ^flagJoinReOrder + } + flag |= flagCollectPredicateColumnsPoint + flag |= flagSyncWaitStatsLoadPoint logic, err := logicalOptimize(ctx, flag, logic) if err != nil { return nil, nil, 0, err @@ -314,6 +327,11 @@ func DoOptimizeAndLogicAsRet(ctx context.Context, sctx sessionctx.Context, flag // DoOptimize optimizes a logical plan to a physical plan. func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (PhysicalPlan, float64, error) { + sessVars := sctx.GetSessionVars() + if sessVars.StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(sctx) + defer debugtrace.LeaveContextCommon(sctx) + } _, finalPlan, cost, err := DoOptimizeAndLogicAsRet(ctx, sctx, flag, logic) return finalPlan, cost, err } @@ -1075,19 +1093,6 @@ func logicalOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (Logic debugtrace.EnterContextCommon(logic.SCtx()) defer debugtrace.LeaveContextCommon(logic.SCtx()) } - // if there is something after flagPrunColumns, do flagPrunColumnsAgain - if flag&flagPrunColumns > 0 && flag-flagPrunColumns > flagPrunColumns { - flag |= flagPrunColumnsAgain - } - if checkStableResultMode(logic.SCtx()) { - flag |= flagStabilizeResults - } - if logic.SCtx().GetSessionVars().StmtCtx.StraightJoinOrder { - // When we use the straight Join Order hint, we should disable the join reorder optimization. - flag &= ^flagJoinReOrder - } - flag |= flagCollectPredicateColumnsPoint - flag |= flagSyncWaitStatsLoadPoint opt := defaultLogicalOptimizeOption() vars := logic.SCtx().GetSessionVars() if vars.StmtCtx.EnableOptimizeTrace { diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 768987edad4dd..bab90db9beb58 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -2495,7 +2495,7 @@ func (p *PhysicalCTE) ExplainID() fmt.Stringer { }) } -// Clone implementes PhysicalPlan interface. +// Clone implements PhysicalPlan interface. func (p *PhysicalCTE) Clone() (PhysicalPlan, error) { cloned := new(PhysicalCTE) base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) @@ -2614,7 +2614,7 @@ func (p *CTEDefinition) MemoryUsage() (sum int64) { type PhysicalCTEStorage PhysicalCTE // ExplainInfo overrides the ExplainInfo -func (p *PhysicalCTEStorage) ExplainInfo() string { +func (*PhysicalCTEStorage) ExplainInfo() string { return "Non-Recursive CTE Storage" } @@ -2688,7 +2688,7 @@ func (p *PhysicalSequence) ExplainID() fmt.Stringer { } // ExplainInfo overrides the ExplainInfo. -func (p *PhysicalSequence) ExplainInfo() string { +func (*PhysicalSequence) ExplainInfo() string { res := "Sequence Node" return res } diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index fded2dd991acb..74c5b68f7a8df 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -669,7 +669,7 @@ func preferKeyColumnFromTable(dataSource *DataSource, originColumns []*expressio // PruneColumns implements the interface of LogicalPlan. // LogicalCTE just do a empty function call. It's logical optimize is indivisual phase. -func (p *LogicalCTE) PruneColumns(parentUsedCols []*expression.Column, opt *logicalOptimizeOp) error { +func (*LogicalCTE) PruneColumns(_ []*expression.Column, _ *logicalOptimizeOp) error { return nil } diff --git a/planner/core/rule_push_down_sequence.go b/planner/core/rule_push_down_sequence.go index 2b29ab0c3a281..9db9314c7f6e3 100644 --- a/planner/core/rule_push_down_sequence.go +++ b/planner/core/rule_push_down_sequence.go @@ -19,7 +19,7 @@ import "context" type pushDownSequenceSolver struct { } -func (pdss *pushDownSequenceSolver) name() string { +func (*pushDownSequenceSolver) name() string { return "push_down_sequence" } diff --git a/planner/core/task.go b/planner/core/task.go index 71075979b7d19..8be55d1882f1c 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -2778,7 +2778,6 @@ func (p *PhysicalSequence) attach2Task(tasks ...task) task { tblColHists: lastTask.tblColHists, } return mppTask - } // mppTask can not : From fa4ecaf2b20f224c4516d43dc08c714697569849 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 24 Apr 2023 18:25:36 +0800 Subject: [PATCH 15/24] fix the cte producer status and add tests --- .../testdata/enforce_mpp_suite_in.json | 23 +- .../testdata/enforce_mpp_suite_out.json | 374 ++++++++++++++---- planner/core/exhaust_physical_plans.go | 33 +- planner/property/physical_property.go | 2 +- 4 files changed, 338 insertions(+), 94 deletions(-) diff --git a/planner/core/casetest/testdata/enforce_mpp_suite_in.json b/planner/core/casetest/testdata/enforce_mpp_suite_in.json index e2b1ad3224b75..e018632f21c7e 100644 --- a/planner/core/casetest/testdata/enforce_mpp_suite_in.json +++ b/planner/core/casetest/testdata/enforce_mpp_suite_in.json @@ -177,13 +177,26 @@ "name": "TestMPPSharedCTEScan", "cases": [ // The most simple case. - "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b ", - "explain with c1 as (select * from t) select c1.* from c1, c1 c2 where c1.b=c2.c", + "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b ", + "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t) select * from c1, c1 c2 where c1.a=c2.b ", + "explain format = 'brief' with c1 as (select * from t) select c1.* from c1, c1 c2 where c1.b=c2.c", // Can work when there's global limit/topn - "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b limit 10", - "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b order by c1.a limit 10", + "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b limit 10", + "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b order by c1.a limit 10", // The c2 references c1, c1 can mpp, and then c2 can mpp, so the main query can mpp. - "explain with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b" + "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b", + // The same SQL, c1 forces to read tikv. So c2 cannot MPP, then the whole SQL. + "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b", + // The two WITH satement can all be MPP. + "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + // The outer one will fail to use MPP. But the inner WITH statement can. But we haven't implemented the least common ancestor to detect the best position of the Sequence. So the whole SQL cannot MPP. + "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + // The inner one will fail. So the whole SQL cannot MPP. + "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select /*+ read_from_storage(tikv[t]) */ * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + // A little change that the inner WITH statement references the outer's c1. + "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from c1) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + // The outer one will fail to use MPP. Since the inner one is references the outer one, the whole SQL cannot MPP. + "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from c1) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a" ] } ] diff --git a/planner/core/casetest/testdata/enforce_mpp_suite_out.json b/planner/core/casetest/testdata/enforce_mpp_suite_out.json index adf4855e7ff4c..f994fc21eff95 100644 --- a/planner/core/casetest/testdata/enforce_mpp_suite_out.json +++ b/planner/core/casetest/testdata/enforce_mpp_suite_out.json @@ -1593,106 +1593,332 @@ "Name": "TestMPPSharedCTEScan", "Cases": [ { - "SQL": "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b ", + "SQL": "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b ", "Plan": [ - "TableReader_70 8000.00 root MppVersion: 1, data:ExchangeSender_69", - "└─ExchangeSender_69 8000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Sequence_68 8000.00 mpp[tiflash] Sequence Node", + "TableReader 8000.00 root MppVersion: 1, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Sequence 8000.00 mpp[tiflash] Sequence Node", " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", - " │ └─Selection_53 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", - " │ └─TableFullScan_21 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " └─HashJoin_54 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", - " ├─ExchangeReceiver_62(Build) 6400.00 mpp[tiflash] ", - " │ └─ExchangeSender_61 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", - " │ └─Selection_56 6400.00 mpp[tiflash] not(isnull(test.t.a))", - " │ └─CTEFullScan_59 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", - " └─Selection_63(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", - " └─CTEFullScan_64 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + " │ └─Selection 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" ], "Warn": null }, { - "SQL": "explain with c1 as (select * from t) select c1.* from c1, c1 c2 where c1.b=c2.c", + "SQL": "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t) select * from c1, c1 c2 where c1.a=c2.b ", "Plan": [ - "TableReader_70 8000.00 root MppVersion: 1, data:ExchangeSender_69", - "└─ExchangeSender_69 8000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_30 8000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", - " └─Sequence_68 8000.00 mpp[tiflash] Sequence Node", + "HashJoin 8000.00 root inner join, equal:[eq(test.t.a, test.t.b)]", + "├─Selection(Build) 6400.00 root not(isnull(test.t.b))", + "│ └─CTEFullScan 8000.00 root CTE:c1 AS c2 data:CTE_0", + "└─Selection(Probe) 6400.00 root not(isnull(test.t.a))", + " └─CTEFullScan 8000.00 root CTE:c1 data:CTE_0", + "CTE_0 8000.00 root Non-Recursive CTE", + "└─Selection(Seed Part) 8000.00 root or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t) select c1.* from c1, c1 c2 where c1.b=c2.c", + "Plan": [ + "TableReader 8000.00 root MppVersion: 1, data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " └─Sequence 8000.00 mpp[tiflash] Sequence Node", " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", - " │ └─Selection_53 8000.00 mpp[tiflash] or(not(isnull(test.t.b)), not(isnull(test.t.c)))", - " │ └─TableFullScan_21 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " └─HashJoin_54 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", - " ├─ExchangeReceiver_62(Build) 6400.00 mpp[tiflash] ", - " │ └─ExchangeSender_61 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", - " │ └─Selection_56 6400.00 mpp[tiflash] not(isnull(test.t.b))", - " │ └─CTEFullScan_59 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", - " └─Selection_63(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.c))", - " └─CTEFullScan_64 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + " │ └─Selection 8000.00 mpp[tiflash] or(not(isnull(test.t.b)), not(isnull(test.t.c)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.c))", + " └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" ], "Warn": null }, { - "SQL": "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b limit 10", + "SQL": "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b limit 10", "Plan": [ - "Limit_36 10.00 root offset:0, count:10", - "└─TableReader_89 10.00 root MppVersion: 1, data:ExchangeSender_88", - " └─ExchangeSender_88 10.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Limit_87 10.00 mpp[tiflash] offset:0, count:10", - " └─Sequence_82 8000.00 mpp[tiflash] Sequence Node", + "Limit 10.00 root offset:0, count:10", + "└─TableReader 10.00 root MppVersion: 1, data:ExchangeSender", + " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit 10.00 mpp[tiflash] offset:0, count:10", + " └─Sequence 8000.00 mpp[tiflash] Sequence Node", " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", - " │ └─Selection_67 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", - " │ └─TableFullScan_24 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " └─HashJoin_83 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", - " ├─ExchangeReceiver_76(Build) 6400.00 mpp[tiflash] ", - " │ └─ExchangeSender_75 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", - " │ └─Selection_70 6400.00 mpp[tiflash] not(isnull(test.t.a))", - " │ └─CTEFullScan_73 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", - " └─Selection_84(Probe) 8.00 mpp[tiflash] not(isnull(test.t.b))", - " └─CTEFullScan_85 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + " │ └─Selection 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" ], "Warn": null }, { - "SQL": "explain with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b order by c1.a limit 10", + "SQL": "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b order by c1.a limit 10", "Plan": [ - "TopN_36 10.00 root test.t.a, offset:0, count:10", - "└─TableReader_80 10.00 root MppVersion: 1, data:ExchangeSender_79", - " └─ExchangeSender_79 10.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TopN_78 10.00 mpp[tiflash] test.t.a, offset:0, count:10", - " └─Sequence_77 8000.00 mpp[tiflash] Sequence Node", + "TopN 10.00 root test.t.a, offset:0, count:10", + "└─TableReader 10.00 root MppVersion: 1, data:ExchangeSender", + " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN 10.00 mpp[tiflash] test.t.a, offset:0, count:10", + " └─Sequence 8000.00 mpp[tiflash] Sequence Node", " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", - " │ └─Selection_62 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", - " │ └─TableFullScan_24 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " └─HashJoin_63 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", - " ├─ExchangeReceiver_71(Build) 6400.00 mpp[tiflash] ", - " │ └─ExchangeSender_70 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", - " │ └─Selection_65 6400.00 mpp[tiflash] not(isnull(test.t.a))", - " │ └─CTEFullScan_68 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", - " └─Selection_72(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", - " └─CTEFullScan_73 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + " │ └─Selection 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b", + "Plan": [ + "TableReader 12500.00 root MppVersion: 1, data:ExchangeSender", + "└─ExchangeSender 12500.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Sequence 12500.00 mpp[tiflash] Sequence Node", + " ├─CTE_0 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " ├─CTE_1 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Projection 10000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " │ └─HashJoin 10000.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", + " │ ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " │ └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0", + " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 AS c1 data:CTE_1", + " └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 data:CTE_1" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b", + "Plan": [ + "HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", + "├─Selection(Build) 8000.00 root not(isnull(test.t.b))", + "│ └─CTEFullScan 10000.00 root CTE:c2 data:CTE_1", + "└─Selection(Probe) 8000.00 root not(isnull(test.t.a))", + " └─CTEFullScan 10000.00 root CTE:c2 AS c1 data:CTE_1", + "CTE_1 10000.00 root Non-Recursive CTE", + "└─HashJoin(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 root CTE:c1 AS c2 data:CTE_0", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─CTEFullScan 10000.00 root CTE:c1 data:CTE_0", + "CTE_0 10000.00 root Non-Recursive CTE", + "└─TableReader(Seed Part) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warn": null + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] }, { - "SQL": "explain with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2 where c1.a=c2.b", - "Plan": [ - "HashJoin_60 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", - "├─Selection_72(Build) 8000.00 root not(isnull(test.t.b))", - "│ └─CTEFullScan_74 10000.00 root CTE:c2 data:CTE_1", - "└─Selection_68(Probe) 8000.00 root not(isnull(test.t.a))", - " └─CTEFullScan_70 10000.00 root CTE:c2 AS c1 data:CTE_1", + "SQL": "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + "Plan": [ + "TableReader 19531.25 root MppVersion: 1, data:ExchangeSender", + "└─ExchangeSender 19531.25 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 19531.25 mpp[tiflash] test.t.a, test.t.b, test.t.c, test.t.a, test.t.b, test.t.c, test.t.a, test.t.b, test.t.c", + " └─Sequence 19531.25 mpp[tiflash] Sequence Node", + " ├─CTE_0 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " ├─CTE_1 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Projection 10000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " │ └─HashJoin 10000.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", + " │ ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " │ └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0", + " └─HashJoin 19531.25 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Sequence 8000.00 mpp[tiflash] Sequence Node", + " │ ├─CTE_2 8000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ │ └─Selection 8000.00 mpp[tiflash] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.c, test.t.b)]", + " │ ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.c))", + " │ │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c3 data:CTE_2", + " │ └─Selection(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c3 AS c4 data:CTE_2", + " └─HashJoin(Probe) 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 AS c1 data:CTE_1", + " └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 data:CTE_1" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + "Plan": [ + "HashJoin 19531.25 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashJoin(Build) 8000.00 root inner join, equal:[eq(test.t.c, test.t.b)]", + "│ ├─Selection(Build) 6400.00 root not(isnull(test.t.b))", + "│ │ └─CTEFullScan 8000.00 root CTE:c3 AS c4 data:CTE_2", + "│ └─Selection(Probe) 6400.00 root not(isnull(test.t.a)), not(isnull(test.t.c))", + "│ └─CTEFullScan 8000.00 root CTE:c3 data:CTE_2", + "└─HashJoin(Probe) 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.b))", + " │ └─CTEFullScan 10000.00 root CTE:c2 data:CTE_1", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.a))", + " └─CTEFullScan 10000.00 root CTE:c2 AS c1 data:CTE_1", + "CTE_2 8000.00 root Non-Recursive CTE", + "└─TableReader(Seed Part) 8000.00 root MppVersion: 1, data:ExchangeSender", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 8000.00 mpp[tiflash] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", "CTE_1 10000.00 root Non-Recursive CTE", - "└─HashJoin_34(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", - " ├─Selection_46(Build) 8000.00 root not(isnull(test.t.c))", - " │ └─CTEFullScan_48 10000.00 root CTE:c1 AS c2 data:CTE_0", - " └─Selection_42(Probe) 8000.00 root not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", - " └─CTEFullScan_44 10000.00 root CTE:c1 data:CTE_0", + "└─HashJoin(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 root CTE:c1 AS c2 data:CTE_0", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─CTEFullScan 10000.00 root CTE:c1 data:CTE_0", "CTE_0 10000.00 root Non-Recursive CTE", - "└─TableReader_29(Seed Part) 10000.00 root MppVersion: 1, data:ExchangeSender_28", - " └─ExchangeSender_28 10000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "└─TableReader(Seed Part) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warn": null + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select /*+ read_from_storage(tikv[t]) */ * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + "Plan": [ + "HashJoin 19531.25 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashJoin(Build) 8000.00 root inner join, equal:[eq(test.t.c, test.t.b)]", + "│ ├─Selection(Build) 6400.00 root not(isnull(test.t.b))", + "│ │ └─CTEFullScan 8000.00 root CTE:c3 AS c4 data:CTE_2", + "│ └─Selection(Probe) 6400.00 root not(isnull(test.t.a)), not(isnull(test.t.c))", + "│ └─CTEFullScan 8000.00 root CTE:c3 data:CTE_2", + "└─HashJoin(Probe) 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.b))", + " │ └─CTEFullScan 10000.00 root CTE:c2 data:CTE_1", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.a))", + " └─CTEFullScan 10000.00 root CTE:c2 AS c1 data:CTE_1", + "CTE_2 8000.00 root Non-Recursive CTE", + "└─Selection(Seed Part) 8000.00 root or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "CTE_1 10000.00 root Non-Recursive CTE", + "└─HashJoin(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 root CTE:c1 AS c2 data:CTE_0", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─CTEFullScan 10000.00 root CTE:c1 data:CTE_0", + "CTE_0 10000.00 root Non-Recursive CTE", + "└─TableReader(Seed Part) 10000.00 root MppVersion: 1, data:ExchangeSender", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] + }, + { + "SQL": "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from c1) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + "Plan": [ + "TableReader 19531.25 root MppVersion: 1, data:ExchangeSender", + "└─ExchangeSender 19531.25 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 19531.25 mpp[tiflash] test.t.a, test.t.b, test.t.c, test.t.a, test.t.b, test.t.c, test.t.a, test.t.b, test.t.c", + " └─Sequence 19531.25 mpp[tiflash] Sequence Node", + " ├─CTE_0 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " ├─CTE_1 10000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Projection 10000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " │ └─HashJoin 10000.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", + " │ ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " │ └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0", + " └─HashJoin 19531.25 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Sequence 8000.00 mpp[tiflash] Sequence Node", + " │ ├─CTE_2 8000.00 mpp[tiflash] Non-Recursive CTE Storage", + " │ │ └─Selection 8000.00 mpp[tiflash] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " │ │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " │ └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.c, test.t.b)]", + " │ ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", + " │ │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.c))", + " │ │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c3 data:CTE_2", + " │ └─Selection(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c3 AS c4 data:CTE_2", + " └─HashJoin(Probe) 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 8000.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 AS c1 data:CTE_1", + " └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 10000.00 mpp[tiflash] CTE:c2 data:CTE_1" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from c1) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", + "Plan": [ + "HashJoin 19531.25 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashJoin(Build) 8000.00 root inner join, equal:[eq(test.t.c, test.t.b)]", + "│ ├─Selection(Build) 6400.00 root not(isnull(test.t.b))", + "│ │ └─CTEFullScan 8000.00 root CTE:c3 AS c4 data:CTE_2", + "│ └─Selection(Probe) 6400.00 root not(isnull(test.t.a)), not(isnull(test.t.c))", + "│ └─CTEFullScan 8000.00 root CTE:c3 data:CTE_2", + "└─HashJoin(Probe) 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.b))", + " │ └─CTEFullScan 10000.00 root CTE:c2 data:CTE_1", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.a))", + " └─CTEFullScan 10000.00 root CTE:c2 AS c1 data:CTE_1", + "CTE_2 8000.00 root Non-Recursive CTE", + "└─Selection(Seed Part) 8000.00 root or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " └─CTEFullScan 10000.00 root CTE:c1 data:CTE_0", + "CTE_1 10000.00 root Non-Recursive CTE", + "└─HashJoin(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─Selection(Build) 8000.00 root not(isnull(test.t.c))", + " │ └─CTEFullScan 10000.00 root CTE:c1 AS c2 data:CTE_0", + " └─Selection(Probe) 8000.00 root not(isnull(test.t.b)), or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " └─CTEFullScan 10000.00 root CTE:c1 data:CTE_0", + "CTE_0 10000.00 root Non-Recursive CTE", + "└─TableReader(Seed Part) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." + ] } ] } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b2a84bac3f1f2..affd5d3424857 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -118,6 +118,8 @@ func (p *PhysicalMergeJoin) tryToGetChildReqProp(prop *property.PhysicalProperty all, desc := prop.AllSameOrder() lProp := property.NewPhysicalProperty(property.RootTaskType, p.LeftJoinKeys, desc, math.MaxFloat64, false) rProp := property.NewPhysicalProperty(property.RootTaskType, p.RightJoinKeys, desc, math.MaxFloat64, false) + lProp.CTEProducerStatus = prop.CTEProducerStatus + rProp.CTEProducerStatus = prop.CTEProducerStatus if !prop.IsSortItemEmpty() { // sort merge join fits the cases of massive ordered data, so desc scan is always expensive. if !all { @@ -440,8 +442,8 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy func (p *LogicalJoin) getHashJoin(prop *property.PhysicalProperty, innerIdx int, useOuterToBuild bool) *PhysicalHashJoin { chReqProps := make([]*property.PhysicalProperty, 2) - chReqProps[innerIdx] = &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64} - chReqProps[1-innerIdx] = &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64} + chReqProps[innerIdx] = &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus} + chReqProps[1-innerIdx] = &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus} if prop.ExpectedCnt < p.stats.RowCount { expCntScale := prop.ExpectedCnt / p.stats.RowCount chReqProps[1-innerIdx].ExpectedCnt = p.children[1-innerIdx].statsInfo().RowCount * expCntScale @@ -484,7 +486,7 @@ func (p *LogicalJoin) constructIndexJoin( return nil } chReqProps := make([]*property.PhysicalProperty, 2) - chReqProps[outerIdx] = &property.PhysicalProperty{TaskTp: property.RootTaskType, ExpectedCnt: math.MaxFloat64, SortItems: prop.SortItems} + chReqProps[outerIdx] = &property.PhysicalProperty{TaskTp: property.RootTaskType, ExpectedCnt: math.MaxFloat64, SortItems: prop.SortItems, CTEProducerStatus: prop.CTEProducerStatus} if prop.ExpectedCnt < p.stats.RowCount { expCntScale := prop.ExpectedCnt / p.stats.RowCount chReqProps[outerIdx].ExpectedCnt = p.children[outerIdx].statsInfo().RowCount * expCntScale @@ -2620,7 +2622,7 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl return ret } -func (lt *LogicalTopN) getPhysLimits(_ *property.PhysicalProperty) []PhysicalPlan { +func (lt *LogicalTopN) getPhysLimits(prop *property.PhysicalProperty) []PhysicalPlan { p, canPass := GetPropByOrderByItems(lt.ByItems) if !canPass { return nil @@ -2632,7 +2634,7 @@ func (lt *LogicalTopN) getPhysLimits(_ *property.PhysicalProperty) []PhysicalPla } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { - resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(lt.Count + lt.Offset), SortItems: p.SortItems} + resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(lt.Count + lt.Offset), SortItems: p.SortItems, CTEProducerStatus: prop.CTEProducerStatus} limit := PhysicalLimit{ Count: lt.Count, Offset: lt.Offset, @@ -2708,8 +2710,8 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ }.Init(la.ctx, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), la.blockOffset, - &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, SortItems: prop.SortItems}, - &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64}) + &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, SortItems: prop.SortItems, CTEProducerStatus: prop.CTEProducerStatus}, + &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus}) apply.SetSchema(la.schema) return []PhysicalPlan{apply}, true, nil } @@ -2840,7 +2842,7 @@ func (lw *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ( var byItems []property.SortItem byItems = append(byItems, lw.PartitionBy...) byItems = append(byItems, lw.OrderBy...) - childProperty := &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, SortItems: byItems, CanAddEnforcer: true} + childProperty := &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, SortItems: byItems, CanAddEnforcer: true, CTEProducerStatus: prop.CTEProducerStatus} if !prop.IsPrefix(childProperty) { return nil, true, nil } @@ -3259,7 +3261,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy hashAggs = append(hashAggs, mppAggs...) } } else { - agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, TaskTp: taskTp}) + agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, TaskTp: taskTp, CTEProducerStatus: prop.CTEProducerStatus}) agg.SetSchema(la.schema.Clone()) hashAggs = append(hashAggs, agg) } @@ -3410,7 +3412,7 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) CTEProducerStatus: prop.CTEProducerStatus, }) } else { - chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: prop.ExpectedCnt, RejectSort: true}) + chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: prop.ExpectedCnt, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus}) } } ua := PhysicalUnionAll{ @@ -3424,6 +3426,7 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ExpectedCnt: prop.ExpectedCnt, TaskTp: property.MppTaskType, RejectSort: true, + CTEProducerStatus: prop.CTEProducerStatus, }) } mppUA := PhysicalUnionAll{mpp: true}.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) @@ -3489,7 +3492,7 @@ func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because operator `MaxOneRow` is not supported now.") return nil, true, nil } - mor := PhysicalMaxOneRow{}.Init(p.ctx, p.stats, p.blockOffset, &property.PhysicalProperty{ExpectedCnt: 2}) + mor := PhysicalMaxOneRow{}.Init(p.ctx, p.stats, p.blockOffset, &property.PhysicalProperty{ExpectedCnt: 2, CTEProducerStatus: prop.CTEProducerStatus}) return []PhysicalPlan{mor}, true, nil } @@ -3507,15 +3510,17 @@ func (p *LogicalCTE) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]Ph func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { possibleChildrenProps := make([][]*property.PhysicalProperty, 0, 2) - childProp := prop.CloneEssentialFields() anyType := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, CanAddEnforcer: true, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} if prop.TaskTp == property.MppTaskType { if prop.CTEProducerStatus == property.SomeCTEFailedMpp { return nil, true, nil } - possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType, childProp}) + anyType.CTEProducerStatus = property.AllCTECanMpp + possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{anyType, prop.CloneEssentialFields()}) } else { - possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{{TaskTp: property.RootTaskType, ExpectedCnt: math.MaxFloat64}, childProp}) + copied := prop.CloneEssentialFields() + copied.CTEProducerStatus = property.SomeCTEFailedMpp + possibleChildrenProps = append(possibleChildrenProps, []*property.PhysicalProperty{{TaskTp: property.RootTaskType, ExpectedCnt: math.MaxFloat64, CTEProducerStatus: property.SomeCTEFailedMpp}, copied}) } if prop.TaskTp != property.MppTaskType && prop.CTEProducerStatus != property.SomeCTEFailedMpp && diff --git a/planner/property/physical_property.go b/planner/property/physical_property.go index e5a289f2302c4..50e05d6d24f60 100644 --- a/planner/property/physical_property.go +++ b/planner/property/physical_property.go @@ -166,7 +166,7 @@ type cteProducerStatus int // Constants for CTE status. const ( - NoCTE cteProducerStatus = iota + NoCTEOrAllProducerCanMPP cteProducerStatus = iota SomeCTEFailedMpp AllCTECanMpp ) From f5d4303139792393cec912e2b886de745e12b7bc Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 26 Apr 2023 02:16:23 +0800 Subject: [PATCH 16/24] address comments --- executor/mpp_gather.go | 6 +++++- planner/core/fragment.go | 3 +++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 075275caa4dc2..0d4525872e699 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -134,7 +134,11 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { } // fixTaskForCTEStorageAndReader fixes the upstream/downstream tasks for the producers and consumers. -// We only setup local transport for the data. +// After we split the fragments. A CTE producer in the fragment will holds all the task address of the consumers. +// For example, the producer has two task on node_1 and node_2. As we know that each consumer also has two task on the same nodes(node_1 and node_2) +// We need to prune address of node_2 for producer's task on node_1 since we just want the producer task on the node_1 only send to the consumer tasks on the node_1. +// And the same for the task on the node_2. +// And the same for the consumer task. We need to prune the unnecessary task address of its producer tasks(i.e. the downstream tasks). func (e *MPPGather) fixTaskForCTEStorageAndReader(exec *tipb.Executor, meta kv.MPPTaskMeta) error { children := make([]*tipb.Executor, 0, 2) switch exec.Tp { diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 5c7705006762a..81a91460b4bb6 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -397,6 +397,9 @@ func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv return tasks, nil } +// flipCTEReader fix the plan tree. Before we enter the func. The plan tree is like ParentPlan->CTEConsumer->ExchangeReceiver. +// The CTEConsumer has no real meaning in MPP's execution. We prune it to make the plan become ParentPlan->ExchangeReceiver. +// But the Recevier needs a schema since itself doesn't hold the schema. So the final plan become ParentPlan->ExchangeRecevier->CTEConsumer. func (f *Fragment) flipCTEReader(currentPlan PhysicalPlan) { newChildren := make([]PhysicalPlan, len(currentPlan.Children())) for i := 0; i < len(currentPlan.Children()); i++ { From 4657c672ff900ca111c8ef9349823b34c62bacde Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 27 Apr 2023 16:07:49 +0800 Subject: [PATCH 17/24] merge the methods --- planner/core/find_best_task.go | 288 ++++++++++----------------------- 1 file changed, 82 insertions(+), 206 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 0a9842de85830..6c5c59ee5cfdd 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -209,144 +209,61 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl prop *property.PhysicalProperty, addEnforcer bool, planCounter *PlanCounterTp, opt *physicalOptimizeOp) (task, int64, error) { var bestTask task = invalidTask var curCntPlan, cntPlan int64 + var err error childTasks := make([]task, 0, len(p.children)) childCnts := make([]int64, len(p.children)) cntPlan = 0 - for _, pp := range physicalPlans { + iteration := func( + selfPhysicalPlan PhysicalPlan, + childTasks []task, + childCnts []int64, + prop *property.PhysicalProperty, + opt *physicalOptimizeOp, + ) ([]task, int64, []int64, error) { // Find best child tasks firstly. childTasks = childTasks[:0] // The curCntPlan records the number of possible plans for pp - curCntPlan = 1 - timeStampNow := p.GetLogicalTS4TaskMap() - savedPlanID := p.ctx.GetSessionVars().PlanID.Load() + curCntPlan := int64(1) for j, child := range p.children { - childProp := pp.GetChildReqProps(j) + childProp := selfPhysicalPlan.GetChildReqProps(j) childTask, cnt, err := child.findBestTask(childProp, &PlanCounterDisabled, opt) childCnts[j] = cnt if err != nil { - return nil, 0, err + return nil, 0, childCnts, err } curCntPlan = curCntPlan * cnt if childTask != nil && childTask.invalid() { - break + return nil, 0, childCnts, nil } childTasks = append(childTasks, childTask) } // This check makes sure that there is no invalid child task. if len(childTasks) != len(p.children) { - continue - } - - // If the target plan can be found in this physicalPlan(pp), rebuild childTasks to build the corresponding combination. - if planCounter.IsForce() && int64(*planCounter) <= curCntPlan { - p.ctx.GetSessionVars().PlanID.Store(savedPlanID) - curCntPlan = int64(*planCounter) - err := p.rebuildChildTasks(&childTasks, pp, childCnts, int64(*planCounter), timeStampNow, opt) - if err != nil { - return nil, 0, err - } - } - - // Combine best child tasks with parent physical plan. - curTask := pp.attach2Task(childTasks...) - - if curTask.invalid() { - continue - } - - // An optimal task could not satisfy the property, so it should be converted here. - if _, ok := curTask.(*rootTask); !ok && prop.TaskTp == property.RootTaskType { - curTask = curTask.convertToRootTask(p.ctx) - } - - // Enforce curTask property - if addEnforcer { - curTask = enforceProperty(prop, curTask, p.basePlan.ctx) - } - - // Optimize by shuffle executor to running in parallel manner. - if _, isMpp := curTask.(*mppTask); !isMpp && prop.IsSortItemEmpty() { - // Currently, we do not regard shuffled plan as a new plan. - curTask = optimizeByShuffle(curTask, p.basePlan.ctx) + return nil, 0, childCnts, nil } + return childTasks, curCntPlan, childCnts, nil + } - cntPlan += curCntPlan - planCounter.Dec(curCntPlan) - - if planCounter.Empty() { - bestTask = curTask - break - } - opt.appendCandidate(p, curTask.plan(), prop) - // Get the most efficient one. - if curIsBetter, err := compareTaskCost(p.ctx, curTask, bestTask, opt); err != nil { - return nil, 0, err - } else if curIsBetter { - bestTask = curTask - } + if seq, ok := p.self.(*LogicalSequence); ok { + iteration = seq.iterateChildPlan } - return bestTask, cntPlan, nil -} -func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, - prop *property.PhysicalProperty, addEnforcer bool, planCounter *PlanCounterTp, opt *physicalOptimizeOp) (task, int64, error) { - var bestTask task = invalidTask - var curCntPlan, cntPlan int64 - childTasks := make([]task, 0, len(p.children)) - childCnts := make([]int64, len(p.children)) - cntPlan = 0 for _, pp := range physicalPlans { - // Find best child tasks firstly. - childTasks = childTasks[:0] - // The curCntPlan records the number of possible plans for pp - curCntPlan = 1 timeStampNow := p.GetLogicalTS4TaskMap() savedPlanID := p.ctx.GetSessionVars().PlanID.Load() - lastIdx := len(p.children) - 1 - for j := 0; j < lastIdx; j++ { - child := p.children[j] - childProp := pp.GetChildReqProps(j) - childTask, cnt, err := child.findBestTask(childProp, &PlanCounterDisabled, opt) - childCnts[j] = cnt - if err != nil { - return nil, 0, err - } - curCntPlan = curCntPlan * cnt - if childTask != nil && childTask.invalid() { - break - } - _, isMpp := childTask.(*mppTask) - if !isMpp && prop.IsFlashProp() { - break - } - childTasks = append(childTasks, childTask) - } - // This check makes sure that there is no invalid child task. - if len(childTasks) != len(p.children)-1 { - continue - } - lastChildProp := pp.GetChildReqProps(lastIdx).CloneEssentialFields() - if lastChildProp.IsFlashProp() { - lastChildProp.CTEProducerStatus = property.AllCTECanMpp - } - lastChildTask, cnt, err := p.Children()[lastIdx].findBestTask(lastChildProp, &PlanCounterDisabled, opt) - childCnts[lastIdx] = cnt + childTasks, curCntPlan, childCnts, err = iteration(pp, childTasks, childCnts, prop, opt) + if err != nil { return nil, 0, err } - curCntPlan = curCntPlan * cnt - if lastChildTask != nil && lastChildTask.invalid() { - continue - } - if _, ok := lastChildTask.(*mppTask); !ok && lastChildProp.CTEProducerStatus == property.AllCTECanMpp { + // This check makes sure that there is no invalid child task. + if len(childTasks) != len(p.children) { continue } - childTasks = append(childTasks, lastChildTask) - // If the target plan can be found in this physicalPlan(pp), rebuild childTasks to build the corresponding combination. if planCounter.IsForce() && int64(*planCounter) <= curCntPlan { p.ctx.GetSessionVars().PlanID.Store(savedPlanID) @@ -398,6 +315,64 @@ func (p *LogicalSequence) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl return bestTask, cntPlan, nil } +// iterateChildPlan does the special part for sequence. We need to iterate its child one by one to check whether the former child is a valid plan and then go to the nex +func (p *LogicalSequence) iterateChildPlan( + selfPhysicalPlan PhysicalPlan, + childTasks []task, + childCnts []int64, + prop *property.PhysicalProperty, + opt *physicalOptimizeOp, +) ([]task, int64, []int64, error) { + // Find best child tasks firstly. + childTasks = childTasks[:0] + // The curCntPlan records the number of possible plans for pp + curCntPlan := int64(1) + lastIdx := len(p.children) - 1 + for j := 0; j < lastIdx; j++ { + child := p.children[j] + childProp := selfPhysicalPlan.GetChildReqProps(j) + childTask, cnt, err := child.findBestTask(childProp, &PlanCounterDisabled, opt) + childCnts[j] = cnt + if err != nil { + return nil, 0, nil, err + } + curCntPlan = curCntPlan * cnt + if childTask != nil && childTask.invalid() { + return nil, 0, nil, nil + } + _, isMpp := childTask.(*mppTask) + if !isMpp && prop.IsFlashProp() { + break + } + childTasks = append(childTasks, childTask) + } + // This check makes sure that there is no invalid child task. + if len(childTasks) != len(p.children)-1 { + return nil, 0, nil, nil + } + + lastChildProp := selfPhysicalPlan.GetChildReqProps(lastIdx).CloneEssentialFields() + if lastChildProp.IsFlashProp() { + lastChildProp.CTEProducerStatus = property.AllCTECanMpp + } + lastChildTask, cnt, err := p.Children()[lastIdx].findBestTask(lastChildProp, &PlanCounterDisabled, opt) + childCnts[lastIdx] = cnt + if err != nil { + return nil, 0, nil, err + } + curCntPlan = curCntPlan * cnt + if lastChildTask != nil && lastChildTask.invalid() { + return nil, 0, nil, nil + } + + if _, ok := lastChildTask.(*mppTask); !ok && lastChildProp.CTEProducerStatus == property.AllCTECanMpp { + return nil, 0, nil, nil + } + + childTasks = append(childTasks, lastChildTask) + return childTasks, curCntPlan, childCnts, nil +} + // compareTaskCost compares cost of curTask and bestTask and returns whether curTask's cost is smaller than bestTask's. func compareTaskCost(_ sessionctx.Context, curTask, bestTask task, op *physicalOptimizeOp) (curIsBetter bool, err error) { curCost, curInvalid, err := getTaskPlanCost(curTask, op) @@ -494,6 +469,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun if prop == nil { return nil, 1, nil } + enumPhysicalPlans4Task := p.enumeratePhysicalPlans4Task // Look up the task with this prop in the task map. // It's used to reduce double counting. bestTask = p.getTask(prop) @@ -558,107 +534,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun var cnt int64 var curTask task - if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp, false, planCounter, opt); err != nil { - return nil, 0, err - } - cntPlan += cnt - if planCounter.Empty() { - goto END - } - - curTask, cnt, err = p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp, true, planCounter, opt) - if err != nil { - return nil, 0, err - } - cntPlan += cnt - if planCounter.Empty() { - bestTask = curTask - goto END - } - opt.appendCandidate(p, curTask.plan(), prop) - if curIsBetter, err := compareTaskCost(p.ctx, curTask, bestTask, opt); err != nil { - return nil, 0, err - } else if curIsBetter { - bestTask = curTask - } - -END: - p.storeTask(prop, bestTask) - return bestTask, cntPlan, nil -} - -// findBestTask implements LogicalPlan interface. -func (p *LogicalSequence) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *physicalOptimizeOp) (bestTask task, cntPlan int64, err error) { - // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, - // and set inner child prop nil, so here we do nothing. - if prop == nil { - return nil, 1, nil - } - // Look up the task with this prop in the task map. - // It's used to reduce double counting. - bestTask = p.getTask(prop) - if bestTask != nil { - planCounter.Dec(1) - return bestTask, 1, nil - } - - canAddEnforcer := prop.CanAddEnforcer - - if prop.TaskTp != property.RootTaskType && !prop.IsFlashProp() { - // Currently all plan cannot totally push down to TiKV. - p.storeTask(prop, invalidTask) - return invalidTask, 0, nil - } - - cntPlan = 0 - // prop should be read only because its cached hashcode might be not consistent - // when it is changed. So we clone a new one for the temporary changes. - newProp := prop.CloneEssentialFields() - var plansFitsProp, plansNeedEnforce []PhysicalPlan - var hintWorksWithProp bool - // Maybe the plan can satisfy the required property, - // so we try to get the task without the enforced sort first. - plansFitsProp, hintWorksWithProp, err = p.self.exhaustPhysicalPlans(newProp) - if err != nil { - return nil, 0, err - } - if !hintWorksWithProp && !newProp.IsSortItemEmpty() { - // If there is a hint in the plan and the hint cannot satisfy the property, - // we enforce this property and try to generate the PhysicalPlan again to - // make sure the hint can work. - canAddEnforcer = true - } - - if canAddEnforcer { - // Then, we use the empty property to get physicalPlans and - // try to get the task with an enforced sort. - newProp.SortItems = []property.SortItem{} - newProp.SortItemsForPartition = []property.SortItem{} - newProp.ExpectedCnt = math.MaxFloat64 - newProp.MPPPartitionCols = nil - newProp.MPPPartitionTp = property.AnyType - var hintCanWork bool - plansNeedEnforce, hintCanWork, err = p.self.exhaustPhysicalPlans(newProp) - if err != nil { - return nil, 0, err - } - if hintCanWork && !hintWorksWithProp { - // If the hint can work with the empty property, but cannot work with - // the required property, we give up `plansFitProp` to make sure the hint - // can work. - plansFitsProp = nil - } - if !hintCanWork && !hintWorksWithProp && !prop.CanAddEnforcer { - // If the original property is not enforced and hint cannot - // work anyway, we give up `plansNeedEnforce` for efficiency, - plansNeedEnforce = nil - } - newProp = prop - } - - var cnt int64 - var curTask task - if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp, false, planCounter, opt); err != nil { + if bestTask, cnt, err = enumPhysicalPlans4Task(plansFitsProp, newProp, false, planCounter, opt); err != nil { return nil, 0, err } cntPlan += cnt @@ -666,7 +542,7 @@ func (p *LogicalSequence) findBestTask(prop *property.PhysicalProperty, planCoun goto END } - curTask, cnt, err = p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp, true, planCounter, opt) + curTask, cnt, err = enumPhysicalPlans4Task(plansNeedEnforce, newProp, true, planCounter, opt) if err != nil { return nil, 0, err } From e5879cdad26f025b6d6acb76d20657382eeb6824 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 10 May 2023 21:11:48 +0800 Subject: [PATCH 18/24] fix the aggregation's bad case --- planner/core/exhaust_physical_plans.go | 24 +++++++++++++++++------- planner/core/logical_plan_builder.go | 2 +- planner/core/logical_plans.go | 1 + planner/core/optimizer.go | 7 +++++++ planner/core/physical_plans.go | 14 ++++++++++++-- planner/core/rule_push_down_sequence.go | 9 +++++++++ 6 files changed, 47 insertions(+), 10 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 3b9b4d42363b7..c7cb56b4cfe1a 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2981,9 +2981,14 @@ func (la *LogicalAggregation) getEnforcedStreamAggs(prop *property.PhysicalPrope *copiedChildProperty = *childProp // It's ok to not deep copy the "cols" field. copiedChildProperty.TaskTp = taskTp + newGbyItems := make([]expression.Expression, len(la.GroupByItems)) + copy(newGbyItems, la.GroupByItems) + newAggFuncs := make([]*aggregation.AggFuncDesc, len(la.AggFuncs)) + copy(newAggFuncs, la.AggFuncs) + agg := basePhysicalAgg{ - GroupByItems: la.GroupByItems, - AggFuncs: la.AggFuncs, + GroupByItems: newGbyItems, + AggFuncs: newAggFuncs, }.initForStream(la.ctx, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), la.blockOffset, copiedChildProperty) agg.SetSchema(la.schema.Clone()) enforcedAggs = append(enforcedAggs, agg) @@ -3060,9 +3065,14 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P *copiedChildProperty = *childProp // It's ok to not deep copy the "cols" field. copiedChildProperty.TaskTp = taskTp + newGbyItems := make([]expression.Expression, len(la.GroupByItems)) + copy(newGbyItems, la.GroupByItems) + newAggFuncs := make([]*aggregation.AggFuncDesc, len(la.AggFuncs)) + copy(newAggFuncs, la.AggFuncs) + agg := basePhysicalAgg{ - GroupByItems: la.GroupByItems, - AggFuncs: la.AggFuncs, + GroupByItems: newGbyItems, + AggFuncs: newAggFuncs, }.initForStream(la.ctx, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), la.blockOffset, copiedChildProperty) agg.SetSchema(la.schema.Clone()) streamAggs = append(streamAggs, agg) @@ -3423,9 +3433,9 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) chReqProps = make([]*property.PhysicalProperty, 0, len(p.children)) for range p.children { chReqProps = append(chReqProps, &property.PhysicalProperty{ - ExpectedCnt: prop.ExpectedCnt, - TaskTp: property.MppTaskType, - RejectSort: true, + ExpectedCnt: prop.ExpectedCnt, + TaskTp: property.MppTaskType, + RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus, }) } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 4d75df188059a..e0ed32f71389e 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4247,7 +4247,7 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L } proj.names = p.OutputNames()[:oldLen] proj.SetSchema(schema) - return proj, nil + return b.tryToBuildSequence(currentLayerCTEs, proj), nil } return b.tryToBuildSequence(currentLayerCTEs, p), nil diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 82488890777bc..cb7d134f12612 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -1705,6 +1705,7 @@ type LogicalLimit struct { Offset uint64 Count uint64 limitHints limitHintInfo + IsPartial bool } // GetPartitionBy returns partition by fields diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 43a75d1000b1a..aa05ee0005a67 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -281,6 +281,9 @@ func checkStableResultMode(sctx sessionctx.Context) bool { // DoOptimizeAndLogicAsRet optimizes a logical plan to a physical plan and return the optimized logical plan. func DoOptimizeAndLogicAsRet(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (LogicalPlan, PhysicalPlan, float64, error) { + if !logic.SCtx().GetSessionVars().InRestrictedSQL { + logutil.BgLogger().Warn("before logical optimize", zap.String("the plan", ToString(logic))) + } sessVars := sctx.GetSessionVars() // if there is something after flagPrunColumns, do flagPrunColumnsAgain if flag&flagPrunColumns > 0 && flag-flagPrunColumns > flagPrunColumns { @@ -300,6 +303,10 @@ func DoOptimizeAndLogicAsRet(ctx context.Context, sctx sessionctx.Context, flag return nil, nil, 0, err } + if !logic.SCtx().GetSessionVars().InRestrictedSQL { + logutil.BgLogger().Warn("after logical optimize", zap.String("the plan", ToString(logic))) + } + if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, nil, 0, errors.Trace(ErrCartesianProductUnsupported) } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index bab90db9beb58..24fef2a0a7a01 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1896,9 +1896,19 @@ func (p *PhysicalHashAgg) MemoryUsage() (sum int64) { // NewPhysicalHashAgg creates a new PhysicalHashAgg from a LogicalAggregation. func NewPhysicalHashAgg(la *LogicalAggregation, newStats *property.StatsInfo, prop *property.PhysicalProperty) *PhysicalHashAgg { + newGbyItems := make([]expression.Expression, len(la.GroupByItems)) + copy(newGbyItems, la.GroupByItems) + newAggFuncs := make([]*aggregation.AggFuncDesc, len(la.AggFuncs)) + // There's some places that rewrites the aggFunc in-place. + // I clone it first. + // It needs a well refactor to make sure that the physical optimize should not change the things of logical plan. + // It's bad for cascades + for i, aggFunc := range la.AggFuncs { + newAggFuncs[i] = aggFunc.Clone() + } agg := basePhysicalAgg{ - GroupByItems: la.GroupByItems, - AggFuncs: la.AggFuncs, + GroupByItems: newGbyItems, + AggFuncs: newAggFuncs, }.initForHash(la.ctx, newStats, la.blockOffset, prop) return agg } diff --git a/planner/core/rule_push_down_sequence.go b/planner/core/rule_push_down_sequence.go index 9db9314c7f6e3..c1d7ac5f44a42 100644 --- a/planner/core/rule_push_down_sequence.go +++ b/planner/core/rule_push_down_sequence.go @@ -28,6 +28,15 @@ func (pdss *pushDownSequenceSolver) optimize(_ context.Context, lp LogicalPlan, } func (pdss *pushDownSequenceSolver) recursiveOptimize(pushedSequence *LogicalSequence, lp LogicalPlan) LogicalPlan { + _, ok := lp.(*LogicalSequence) + if !ok && pushedSequence == nil { + newChildren := make([]LogicalPlan, 0, len(lp.Children())) + for _, child := range lp.Children() { + newChildren = append(newChildren, pdss.recursiveOptimize(nil, child)) + } + lp.SetChildren(newChildren...) + return lp + } switch x := lp.(type) { case *LogicalSequence: if pushedSequence == nil { From 84e8ac8b9cb80254ddc224546a87b923fbda9d4e Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 17 May 2023 17:55:34 +0800 Subject: [PATCH 19/24] address comments --- go.mod | 3 +-- go.sum | 4 ++-- planner/core/find_best_task.go | 5 ++--- planner/core/fragment.go | 2 +- 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/go.mod b/go.mod index 04a54caaf2bf7..6cd208be57461 100644 --- a/go.mod +++ b/go.mod @@ -78,7 +78,7 @@ require ( github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e - github.com/pingcap/tipb v0.0.0-20230427024529-aed92caf20b9 + github.com/pingcap/tipb v0.0.0-20230516140330-b3e432c40cb3 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.15.1 github.com/prometheus/client_model v0.4.0 @@ -286,6 +286,5 @@ replace ( github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible github.com/go-ldap/ldap/v3 => github.com/YangKeao/ldap/v3 v3.4.5-0.20230421065457-369a3bab1117 github.com/pingcap/tidb/parser => ./parser - github.com/pingcap/tipb => github.com/pingcap/tipb v0.0.0-20230509143956-5c1ce6aa17a0 go.opencensus.io => go.opencensus.io v0.23.1-0.20220331163232-052120675fac ) diff --git a/go.sum b/go.sum index 26ade81746e7e..dcbe2a162ef15 100644 --- a/go.sum +++ b/go.sum @@ -793,8 +793,8 @@ github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tipb v0.0.0-20230509143956-5c1ce6aa17a0 h1:mOAPx5exmo6QACzFRzuheW77kOufOrXXde8FdwgbiTM= -github.com/pingcap/tipb v0.0.0-20230509143956-5c1ce6aa17a0/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20230516140330-b3e432c40cb3 h1:ZVehx2Mand1frpxzJud7FUOonbLZeXXQpEsNdjnEAJA= +github.com/pingcap/tipb v0.0.0-20230516140330-b3e432c40cb3/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 44f3ee5efa2ae..54365cf7a529d 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -480,7 +480,6 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun if prop == nil { return nil, 1, nil } - enumPhysicalPlans4Task := p.enumeratePhysicalPlans4Task // Look up the task with this prop in the task map. // It's used to reduce double counting. bestTask = p.getTask(prop) @@ -545,7 +544,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun var cnt int64 var curTask task - if bestTask, cnt, err = enumPhysicalPlans4Task(plansFitsProp, newProp, false, planCounter, opt); err != nil { + if bestTask, cnt, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp, false, planCounter, opt); err != nil { return nil, 0, err } cntPlan += cnt @@ -553,7 +552,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun goto END } - curTask, cnt, err = enumPhysicalPlans4Task(plansNeedEnforce, newProp, true, planCounter, opt) + curTask, cnt, err = p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp, true, planCounter, opt) if err != nil { return nil, 0, err } diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 81a91460b4bb6..104e83ede01e3 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -399,7 +399,7 @@ func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv // flipCTEReader fix the plan tree. Before we enter the func. The plan tree is like ParentPlan->CTEConsumer->ExchangeReceiver. // The CTEConsumer has no real meaning in MPP's execution. We prune it to make the plan become ParentPlan->ExchangeReceiver. -// But the Recevier needs a schema since itself doesn't hold the schema. So the final plan become ParentPlan->ExchangeRecevier->CTEConsumer. +// But the Receiver needs a schema since itself doesn't hold the schema. So the final plan become ParentPlan->ExchangeReceiver->CTEConsumer. func (f *Fragment) flipCTEReader(currentPlan PhysicalPlan) { newChildren := make([]PhysicalPlan, len(currentPlan.Children())) for i := 0; i < len(currentPlan.Children()); i++ { From 9004ef0f99bcfd32e732e754f02106a75e120dfa Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 17 May 2023 18:07:43 +0800 Subject: [PATCH 20/24] fix bazel_prepare --- DEPS.bzl | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 4d25dcfaba728..2f54a47f57777 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -3450,9 +3450,8 @@ def go_deps(): name = "com_github_pingcap_tipb", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/tipb", - replace = "github.com/pingcap/tipb", - sum = "h1:mOAPx5exmo6QACzFRzuheW77kOufOrXXde8FdwgbiTM=", - version = "v0.0.0-20230509143956-5c1ce6aa17a0", + sum = "h1:ZVehx2Mand1frpxzJud7FUOonbLZeXXQpEsNdjnEAJA=", + version = "v0.0.0-20230516140330-b3e432c40cb3", ) go_repository( name = "com_github_pkg_browser", From 9088ad448467b64767ead73aeed27a7138abc7c9 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 18 May 2023 17:27:57 +0800 Subject: [PATCH 21/24] remove debug log --- planner/core/optimizer.go | 7 ------- 1 file changed, 7 deletions(-) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index aa05ee0005a67..43a75d1000b1a 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -281,9 +281,6 @@ func checkStableResultMode(sctx sessionctx.Context) bool { // DoOptimizeAndLogicAsRet optimizes a logical plan to a physical plan and return the optimized logical plan. func DoOptimizeAndLogicAsRet(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (LogicalPlan, PhysicalPlan, float64, error) { - if !logic.SCtx().GetSessionVars().InRestrictedSQL { - logutil.BgLogger().Warn("before logical optimize", zap.String("the plan", ToString(logic))) - } sessVars := sctx.GetSessionVars() // if there is something after flagPrunColumns, do flagPrunColumnsAgain if flag&flagPrunColumns > 0 && flag-flagPrunColumns > flagPrunColumns { @@ -303,10 +300,6 @@ func DoOptimizeAndLogicAsRet(ctx context.Context, sctx sessionctx.Context, flag return nil, nil, 0, err } - if !logic.SCtx().GetSessionVars().InRestrictedSQL { - logutil.BgLogger().Warn("after logical optimize", zap.String("the plan", ToString(logic))) - } - if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, nil, 0, errors.Trace(ErrCartesianProductUnsupported) } From 06f4b3b6c251e3763ab8447d917c60cc32ff596c Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 23 May 2023 20:53:35 +0800 Subject: [PATCH 22/24] address comments --- planner/core/access_object.go | 6 ++++-- planner/core/fragment.go | 5 ++++- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/planner/core/access_object.go b/planner/core/access_object.go index 40c938a83e5d0..5c36226d7bcb2 100644 --- a/planner/core/access_object.go +++ b/planner/core/access_object.go @@ -230,6 +230,9 @@ func (o OtherAccessObject) SetIntoPB(pb *tipb.ExplainOperator) { if pb == nil { return } + if o == "" { + return + } pb.AccessObjects = []*tipb.AccessObject{ { AccessObject: &tipb.AccessObject_OtherObject{OtherObject: string(o)}, @@ -398,8 +401,7 @@ func (p *PhysicalTableReader) accessObject(sctx sessionctx.Context) AccessObject if len(p.PartitionInfos) == 0 { ts, ok := p.TablePlans[0].(*PhysicalTableScan) if !ok { - cte := p.TablePlans[0].(*PhysicalCTE) - return OtherAccessObject(fmt.Sprintf("cte: %v as %v", cte.cteName, cte.cteAsName)) + return OtherAccessObject("") } asName := "" if ts.TableAsName != nil && len(ts.TableAsName.O) > 0 { diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 104e83ede01e3..97d5659728fff 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -397,7 +397,7 @@ func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv return tasks, nil } -// flipCTEReader fix the plan tree. Before we enter the func. The plan tree is like ParentPlan->CTEConsumer->ExchangeReceiver. +// flipCTEReader fix the plan tree. In the func generateTasksForCTEReader, we create the plan tree like ParentPlan->CTEConsumer->ExchangeReceiver. // The CTEConsumer has no real meaning in MPP's execution. We prune it to make the plan become ParentPlan->ExchangeReceiver. // But the Receiver needs a schema since itself doesn't hold the schema. So the final plan become ParentPlan->ExchangeReceiver->CTEConsumer. func (f *Fragment) flipCTEReader(currentPlan PhysicalPlan) { @@ -416,6 +416,9 @@ func (f *Fragment) flipCTEReader(currentPlan PhysicalPlan) { currentPlan.SetChildren(newChildren...) } +// genereateTasksForCTEReader generates the task leaf for cte reader. +// A fragment's leaf must be Exchange and we could not lost the information of the CTE. +// So we create the plan like ParentPlan->CTEReader->ExchangeReceiver. func (e *mppTaskGenerator) generateTasksForCTEReader(cteReader *PhysicalCTE) (err error) { group := e.CTEGroups[cteReader.CTE.IDForStorage] if group.StorageFragments == nil { From 51a117836c8cccbad868a7ee98fdb548818bbb6a Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 23 May 2023 21:14:37 +0800 Subject: [PATCH 23/24] fix lint --- planner/core/stats.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/stats.go b/planner/core/stats.go index f43b6a8666363..4caa3a2feb293 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -1095,7 +1095,7 @@ func (p *LogicalWindow) ExtractColGroups(colGroups [][]*expression.Column) [][]* } // DeriveStats implement LogicalPlan DeriveStats interface. -func (p *LogicalCTE) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { +func (p *LogicalCTE) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { if p.stats != nil { return p.stats, nil } From 34864c8738e089596777c1cfc00b9048a9c0a651 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 24 May 2023 20:42:01 +0800 Subject: [PATCH 24/24] fix tests --- .../testdata/enforce_mpp_suite_out.json | 174 +++++++++--------- 1 file changed, 87 insertions(+), 87 deletions(-) diff --git a/planner/core/casetest/testdata/enforce_mpp_suite_out.json b/planner/core/casetest/testdata/enforce_mpp_suite_out.json index e1998eb9cc426..85fc7ca3983b6 100644 --- a/planner/core/casetest/testdata/enforce_mpp_suite_out.json +++ b/planner/core/casetest/testdata/enforce_mpp_suite_out.json @@ -1598,33 +1598,33 @@ { "SQL": "explain format = 'brief' with c1 as (select * from t) select * from c1, c1 c2 where c1.a=c2.b ", "Plan": [ - "TableReader 8000.00 root MppVersion: 1, data:ExchangeSender", - "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Sequence 8000.00 mpp[tiflash] Sequence Node", - " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", - " │ └─Selection 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", - " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", - " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", - " │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.a))", - " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", - " └─Selection(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", - " └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + "TableReader 9999.99 root MppVersion: 1, data:ExchangeSender", + "└─ExchangeSender 9999.99 mpp[tiflash] ExchangeType: PassThrough", + " └─Sequence 9999.99 mpp[tiflash] Sequence Node", + " ├─CTE_0 9999.99 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection 9999.99 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin 9999.99 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 7999.99 mpp[tiflash] ", + " │ └─ExchangeSender 7999.99 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 7999.99 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 7999.99 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" ], "Warn": null }, { "SQL": "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t) select * from c1, c1 c2 where c1.a=c2.b ", "Plan": [ - "HashJoin 8000.00 root inner join, equal:[eq(test.t.a, test.t.b)]", - "├─Selection(Build) 6400.00 root not(isnull(test.t.b))", - "│ └─CTEFullScan 8000.00 root CTE:c1 AS c2 data:CTE_0", - "└─Selection(Probe) 6400.00 root not(isnull(test.t.a))", - " └─CTEFullScan 8000.00 root CTE:c1 data:CTE_0", - "CTE_0 8000.00 root Non-Recursive CTE", - "└─Selection(Seed Part) 8000.00 root or(not(isnull(test.t.a)), not(isnull(test.t.b)))", - " └─TableReader 10000.00 root data:TableFullScan", + "HashJoin 9999.99 root inner join, equal:[eq(test.t.a, test.t.b)]", + "├─Selection(Build) 7999.99 root not(isnull(test.t.b))", + "│ └─CTEFullScan 9999.99 root CTE:c1 AS c2 data:CTE_0", + "└─Selection(Probe) 7999.99 root not(isnull(test.t.a))", + " └─CTEFullScan 9999.99 root CTE:c1 data:CTE_0", + "CTE_0 9999.99 root Non-Recursive CTE", + "└─TableReader(Seed Part) 9999.99 root data:Selection", + " └─Selection 9999.99 cop[tikv] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warn": [ @@ -1634,20 +1634,20 @@ { "SQL": "explain format = 'brief' with c1 as (select * from t) select c1.* from c1, c1 c2 where c1.b=c2.c", "Plan": [ - "TableReader 8000.00 root MppVersion: 1, data:ExchangeSender", - "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", - " └─Sequence 8000.00 mpp[tiflash] Sequence Node", - " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", - " │ └─Selection 8000.00 mpp[tiflash] or(not(isnull(test.t.b)), not(isnull(test.t.c)))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", - " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", - " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", - " │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.b))", - " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", - " └─Selection(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.c))", - " └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + "TableReader 9999.99 root MppVersion: 1, data:ExchangeSender", + "└─ExchangeSender 9999.99 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 9999.99 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " └─Sequence 9999.99 mpp[tiflash] Sequence Node", + " ├─CTE_0 9999.99 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection 9999.99 mpp[tiflash] or(not(isnull(test.t.b)), not(isnull(test.t.c)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin 9999.99 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.c)]", + " ├─ExchangeReceiver(Build) 7999.99 mpp[tiflash] ", + " │ └─ExchangeSender 7999.99 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 7999.99 mpp[tiflash] not(isnull(test.t.b))", + " │ └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 7999.99 mpp[tiflash] not(isnull(test.t.c))", + " └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" ], "Warn": null }, @@ -1658,17 +1658,17 @@ "└─TableReader 10.00 root MppVersion: 1, data:ExchangeSender", " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", " └─Limit 10.00 mpp[tiflash] offset:0, count:10", - " └─Sequence 8000.00 mpp[tiflash] Sequence Node", - " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", - " │ └─Selection 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", - " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", - " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", - " │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.a))", - " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Sequence 9999.99 mpp[tiflash] Sequence Node", + " ├─CTE_0 9999.99 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection 9999.99 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin 9999.99 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 7999.99 mpp[tiflash] ", + " │ └─ExchangeSender 7999.99 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 7999.99 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 data:CTE_0", " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.t.b))", - " └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + " └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" ], "Warn": null }, @@ -1679,17 +1679,17 @@ "└─TableReader 10.00 root MppVersion: 1, data:ExchangeSender", " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", " └─TopN 10.00 mpp[tiflash] test.t.a, offset:0, count:10", - " └─Sequence 8000.00 mpp[tiflash] Sequence Node", - " ├─CTE_0 8000.00 mpp[tiflash] Non-Recursive CTE Storage", - " │ └─Selection 8000.00 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", - " ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", - " │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", - " │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.a))", - " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 data:CTE_0", - " └─Selection(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", - " └─CTEFullScan 8000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" + " └─Sequence 9999.99 mpp[tiflash] Sequence Node", + " ├─CTE_0 9999.99 mpp[tiflash] Non-Recursive CTE Storage", + " │ └─Selection 9999.99 mpp[tiflash] or(not(isnull(test.t.a)), not(isnull(test.t.b)))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " └─HashJoin 9999.99 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", + " ├─ExchangeReceiver(Build) 7999.99 mpp[tiflash] ", + " │ └─ExchangeSender 7999.99 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 7999.99 mpp[tiflash] not(isnull(test.t.a))", + " │ └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 data:CTE_0", + " └─Selection(Probe) 7999.99 mpp[tiflash] not(isnull(test.t.b))", + " └─CTEFullScan 9999.99 mpp[tiflash] CTE:c1 AS c2 data:CTE_0" ], "Warn": null }, @@ -1761,19 +1761,19 @@ " │ └─Selection(Probe) 8000.00 mpp[tiflash] not(isnull(test.t.c))", " │ └─CTEFullScan 10000.00 mpp[tiflash] CTE:c1 AS c2 data:CTE_0", " └─HashJoin 19531.25 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", - " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", - " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", - " │ └─Sequence 8000.00 mpp[tiflash] Sequence Node", - " │ ├─CTE_2 8000.00 mpp[tiflash] Non-Recursive CTE Storage", - " │ │ └─Selection 8000.00 mpp[tiflash] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", - " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", - " │ └─HashJoin 8000.00 mpp[tiflash] inner join, equal:[eq(test.t.c, test.t.b)]", - " │ ├─ExchangeReceiver(Build) 6400.00 mpp[tiflash] ", - " │ │ └─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", - " │ │ └─Selection 6400.00 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.c))", - " │ │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c3 data:CTE_2", - " │ └─Selection(Probe) 6400.00 mpp[tiflash] not(isnull(test.t.b))", - " │ └─CTEFullScan 8000.00 mpp[tiflash] CTE:c3 AS c4 data:CTE_2", + " ├─ExchangeReceiver(Build) 9999.98 mpp[tiflash] ", + " │ └─ExchangeSender 9999.98 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Sequence 9999.98 mpp[tiflash] Sequence Node", + " │ ├─CTE_2 9999.98 mpp[tiflash] Non-Recursive CTE Storage", + " │ │ └─Selection 9999.98 mpp[tiflash] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", + " │ └─HashJoin 9999.98 mpp[tiflash] inner join, equal:[eq(test.t.c, test.t.b)]", + " │ ├─ExchangeReceiver(Build) 7999.98 mpp[tiflash] ", + " │ │ └─ExchangeSender 7999.98 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ │ └─Selection 7999.98 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.c))", + " │ │ └─CTEFullScan 9999.98 mpp[tiflash] CTE:c3 data:CTE_2", + " │ └─Selection(Probe) 7999.98 mpp[tiflash] not(isnull(test.t.b))", + " │ └─CTEFullScan 9999.98 mpp[tiflash] CTE:c3 AS c4 data:CTE_2", " └─HashJoin(Probe) 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.b)]", " ├─ExchangeReceiver(Build) 8000.00 mpp[tiflash] ", " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", @@ -1788,21 +1788,21 @@ "SQL": "explain format = 'brief' with c1 as (select /*+ read_from_storage(tikv[t]) */ * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", "Plan": [ "HashJoin 19531.25 root inner join, equal:[eq(test.t.a, test.t.a)]", - "├─HashJoin(Build) 8000.00 root inner join, equal:[eq(test.t.c, test.t.b)]", - "│ ├─Selection(Build) 6400.00 root not(isnull(test.t.b))", - "│ │ └─CTEFullScan 8000.00 root CTE:c3 AS c4 data:CTE_2", - "│ └─Selection(Probe) 6400.00 root not(isnull(test.t.a)), not(isnull(test.t.c))", - "│ └─CTEFullScan 8000.00 root CTE:c3 data:CTE_2", + "├─HashJoin(Build) 9999.98 root inner join, equal:[eq(test.t.c, test.t.b)]", + "│ ├─Selection(Build) 7999.98 root not(isnull(test.t.b))", + "│ │ └─CTEFullScan 9999.98 root CTE:c3 AS c4 data:CTE_2", + "│ └─Selection(Probe) 7999.98 root not(isnull(test.t.a)), not(isnull(test.t.c))", + "│ └─CTEFullScan 9999.98 root CTE:c3 data:CTE_2", "└─HashJoin(Probe) 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", " ├─Selection(Build) 8000.00 root not(isnull(test.t.b))", " │ └─CTEFullScan 10000.00 root CTE:c2 data:CTE_1", " └─Selection(Probe) 8000.00 root not(isnull(test.t.a))", " └─CTEFullScan 10000.00 root CTE:c2 AS c1 data:CTE_1", - "CTE_2 8000.00 root Non-Recursive CTE", - "└─TableReader(Seed Part) 8000.00 root MppVersion: 1, data:ExchangeSender", - " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Selection 8000.00 mpp[tiflash] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + "CTE_2 9999.98 root Non-Recursive CTE", + "└─TableReader(Seed Part) 9999.98 root MppVersion: 1, data:ExchangeSender", + " └─ExchangeSender 9999.98 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 9999.98 mpp[tiflash] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo", "CTE_1 10000.00 root Non-Recursive CTE", "└─HashJoin(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]", " ├─Selection(Build) 8000.00 root not(isnull(test.t.c))", @@ -1821,19 +1821,19 @@ "SQL": "explain format = 'brief' with c1 as (select * from t), c2 as (select c1.* from c1, c1 c2 where c1.b=c2.c) select * from c2 c1, c2, (with c3 as (select /*+ read_from_storage(tikv[t]) */ * from t) select c3.* from c3, c3 c4 where c3.c=c4.b) c3 where c1.a=c2.b and c1.a=c3.a", "Plan": [ "HashJoin 19531.25 root inner join, equal:[eq(test.t.a, test.t.a)]", - "├─HashJoin(Build) 8000.00 root inner join, equal:[eq(test.t.c, test.t.b)]", - "│ ├─Selection(Build) 6400.00 root not(isnull(test.t.b))", - "│ │ └─CTEFullScan 8000.00 root CTE:c3 AS c4 data:CTE_2", - "│ └─Selection(Probe) 6400.00 root not(isnull(test.t.a)), not(isnull(test.t.c))", - "│ └─CTEFullScan 8000.00 root CTE:c3 data:CTE_2", + "├─HashJoin(Build) 9999.98 root inner join, equal:[eq(test.t.c, test.t.b)]", + "│ ├─Selection(Build) 7999.98 root not(isnull(test.t.b))", + "│ │ └─CTEFullScan 9999.98 root CTE:c3 AS c4 data:CTE_2", + "│ └─Selection(Probe) 7999.98 root not(isnull(test.t.a)), not(isnull(test.t.c))", + "│ └─CTEFullScan 9999.98 root CTE:c3 data:CTE_2", "└─HashJoin(Probe) 12500.00 root inner join, equal:[eq(test.t.a, test.t.b)]", " ├─Selection(Build) 8000.00 root not(isnull(test.t.b))", " │ └─CTEFullScan 10000.00 root CTE:c2 data:CTE_1", " └─Selection(Probe) 8000.00 root not(isnull(test.t.a))", " └─CTEFullScan 10000.00 root CTE:c2 AS c1 data:CTE_1", - "CTE_2 8000.00 root Non-Recursive CTE", - "└─Selection(Seed Part) 8000.00 root or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", - " └─TableReader 10000.00 root data:TableFullScan", + "CTE_2 9999.98 root Non-Recursive CTE", + "└─TableReader(Seed Part) 9999.98 root data:Selection", + " └─Selection 9999.98 cop[tikv] or(and(not(isnull(test.t.c)), not(isnull(test.t.a))), not(isnull(test.t.b)))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "CTE_1 10000.00 root Non-Recursive CTE", "└─HashJoin(Seed Part) 10000.00 root inner join, equal:[eq(test.t.b, test.t.c)]",