From 5e9247acb3c9a94a3d3a02f6f0f3cd6ae9a978dd Mon Sep 17 00:00:00 2001 From: yisaer Date: Tue, 26 Jul 2022 14:32:51 +0800 Subject: [PATCH 1/6] support cost detail framework Signed-off-by: yisaer --- planner/core/common_plans.go | 7 +- planner/core/find_best_task.go | 36 ++++++++- planner/core/optimizer.go | 5 +- planner/core/plan.go | 31 ++++++- planner/core/plan_cost.go | 134 ++++++++++++++++++------------- planner/core/plan_cost_detail.go | 35 ++++++++ util/tracing/opt_trace.go | 34 ++++++++ 7 files changed, 220 insertions(+), 62 deletions(-) create mode 100644 planner/core/plan_cost_detail.go diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 97820b5ec2cb1..7efb7510a8a60 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -1239,7 +1239,8 @@ func (e *Explain) RenderResult() error { if e.Analyze && strings.ToLower(e.Format) == types.ExplainFormatTrueCardCost { pp, ok := e.TargetPlan.(PhysicalPlan) if ok { - if _, err := pp.GetPlanCost(property.RootTaskType, CostFlagRecalculate|CostFlagUseTrueCardinality); err != nil { + if _, err := pp.GetPlanCost(property.RootTaskType, + NewDefaultPlanCostOption().WithCostFlag(CostFlagRecalculate|CostFlagUseTrueCardinality)); err != nil { return err } } else { @@ -1409,7 +1410,7 @@ func (e *Explain) getOperatorInfo(p Plan, id string) (string, string, string, st estCost := "N/A" if pp, ok := p.(PhysicalPlan); ok { if p.SCtx().GetSessionVars().EnableNewCostInterface { - planCost, _ := pp.GetPlanCost(property.RootTaskType, 0) + planCost, _ := pp.GetPlanCost(property.RootTaskType, NewDefaultPlanCostOption()) estCost = strconv.FormatFloat(planCost, 'f', 2, 64) } else { estCost = strconv.FormatFloat(pp.Cost(), 'f', 2, 64) @@ -1519,7 +1520,7 @@ func binaryOpFromFlatOp(explainCtx sessionctx.Context, op *FlatOperator, out *ti if op.IsPhysicalPlan { p := op.Origin.(PhysicalPlan) if p.SCtx().GetSessionVars().EnableNewCostInterface { - out.Cost, _ = p.GetPlanCost(property.RootTaskType, 0) + out.Cost, _ = p.GetPlanCost(property.RootTaskType, NewDefaultPlanCostOption()) } else { out.Cost = p.Cost() } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 1d97e73cf8541..9c382b27024e0 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -329,13 +329,15 @@ func getTaskPlanCost(t task) (float64, bool, error) { default: return 0, false, errors.New("unknown task type") } - cost, err := t.plan().GetPlanCost(taskType, 0) + cost, err := t.plan().GetPlanCost(taskType, NewDefaultPlanCostOption()) return cost, false, err } type physicalOptimizeOp struct { // tracer is goring to track optimize steps during physical optimizing tracer *tracing.PhysicalOptimizeTracer + + currCostDetail *tracing.PhysicalPlanCostDetail } func defaultPhysicalOptimizeOption() *physicalOptimizeOp { @@ -359,6 +361,34 @@ func (op *physicalOptimizeOp) appendCandidate(lp LogicalPlan, pp PhysicalPlan, p pp.appendChildCandidate(op) } +func (op *physicalOptimizeOp) appendPlanCostDetail(pp PhysicalPlan) *physicalOptimizeOp { + if op == nil || op.tracer == nil { + return nil + } + if op.currCostDetail != nil { + op.tracer.PhysicalPlanCostDetails[pp.ID()] = op.currCostDetail + op.currCostDetail = nil + } + op.currCostDetail = tracing.NewPhysicalPlanCostDetail(pp.ID(), pp.TP()) + return op +} + +func (op *physicalOptimizeOp) appendParamCostDetail(k string, v interface{}) *physicalOptimizeOp { + if op == nil || op.tracer == nil || op.currCostDetail == nil { + return nil + } + op.currCostDetail.AddParam(k, v) + return op +} + +func (op *physicalOptimizeOp) appendCurrCostDesc(desc string) *physicalOptimizeOp { + if op == nil || op.tracer == nil || op.currCostDetail == nil { + return nil + } + op.currCostDetail.SetDesc(desc) + return op +} + // findBestTask implements LogicalPlan interface. func (p *baseLogicalPlan) 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, @@ -2018,7 +2048,7 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida pointGetPlan.UnsignedHandle = mysql.HasUnsignedFlag(ds.handleCols.GetCol(0).RetType.GetFlag()) pointGetPlan.PartitionInfo = partitionInfo pointGetPlan.accessCols = ds.TblCols - cost = pointGetPlan.GetCost() + cost = pointGetPlan.GetCost(opt) // Add filter condition to table plan now. if len(candidate.path.TableFilters) > 0 { sessVars := ds.ctx.GetSessionVars() @@ -2040,7 +2070,7 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida } else { pointGetPlan.accessCols = ds.TblCols } - cost = pointGetPlan.GetCost() + cost = pointGetPlan.GetCost(opt) // Add index condition to table plan now. if len(candidate.path.IndexFilters)+len(candidate.path.TableFilters) > 0 { sessVars := ds.ctx.GetSessionVars() diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index d69a9d99ece0b..1721e4dd398dc 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -629,7 +629,10 @@ func physicalOptimize(logic LogicalPlan, planCounter *PlanCounterTp) (plan Physi opt := defaultPhysicalOptimizeOption() stmtCtx := logic.SCtx().GetSessionVars().StmtCtx if stmtCtx.EnableOptimizeTrace { - tracer := &tracing.PhysicalOptimizeTracer{Candidates: make(map[int]*tracing.CandidatePlanTrace)} + tracer := &tracing.PhysicalOptimizeTracer{ + PhysicalPlanCostDetails: make(map[int]*tracing.PhysicalPlanCostDetail), + Candidates: make(map[int]*tracing.CandidatePlanTrace), + } opt = opt.withEnableOptimizeTracer(tracer) defer func() { if err == nil { diff --git a/planner/core/plan.go b/planner/core/plan.go index 5eede1314c5ab..8c12c76bf6eb1 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -329,7 +329,7 @@ type PhysicalPlan interface { Plan // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. - GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) + GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) // attach2Task makes the current physical plan as the father of task's physicalPlan and updates the cost of // current task. If the child's task is cop task, some operator may close this task and return a new rootTask. @@ -381,6 +381,35 @@ type PhysicalPlan interface { appendChildCandidate(op *physicalOptimizeOp) } +// NewDefaultPlanCostOption returns PlanCostOption +func NewDefaultPlanCostOption() *PlanCostOption { + return &PlanCostOption{} +} + +// PlanCostOption indicates option during GetPlanCost +type PlanCostOption struct { + CostFlag uint64 + tracer *physicalOptimizeOp +} + +// WithCostFlag set costflag +func (op *PlanCostOption) WithCostFlag(flag uint64) *PlanCostOption { + if op == nil { + return nil + } + op.CostFlag = flag + return op +} + +// WithOptimizeTracer set tracer +func (op *PlanCostOption) WithOptimizeTracer(tracer *physicalOptimizeOp) *PlanCostOption { + if op == nil { + return nil + } + op.tracer = tracer + return op +} + type baseLogicalPlan struct { basePlan diff --git a/planner/core/plan_cost.go b/planner/core/plan_cost.go index 250f01dfdacba..f94f1d093ef2d 100644 --- a/planner/core/plan_cost.go +++ b/planner/core/plan_cost.go @@ -45,14 +45,15 @@ func hasCostFlag(costFlag, flag uint64) bool { } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *basePhysicalPlan) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *basePhysicalPlan) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { // just calculate the cost once and always reuse it return p.planCost, nil } p.planCost = 0 // the default implementation, the operator have no cost for _, child := range p.children { - childCost, err := child.GetPlanCost(taskType, costFlag) + childCost, err := child.GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -63,7 +64,8 @@ func (p *basePhysicalPlan) GetPlanCost(taskType property.TaskType, costFlag uint } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalSelection) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalSelection) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } @@ -99,7 +101,7 @@ func (p *PhysicalSelection) GetPlanCost(taskType property.TaskType, costFlag uin selfCost = getCardinality(p.children[0], costFlag) * float64(len(p.Conditions)) * cpuFactor } - childCost, err := p.children[0].GetPlanCost(taskType, costFlag) + childCost, err := p.children[0].GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -122,11 +124,12 @@ func (p *PhysicalProjection) GetCost(count float64) float64 { } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalProjection) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalProjection) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } - childCost, err := p.children[0].GetPlanCost(taskType, costFlag) + childCost, err := p.children[0].GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -183,14 +186,15 @@ func (p *PhysicalIndexLookUpReader) GetCost(costFlag uint64) (cost float64) { } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalIndexLookUpReader) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalIndexLookUpReader) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } p.planCost = 0 // child's cost for _, child := range []PhysicalPlan{p.indexPlan, p.tablePlan} { - childCost, err := child.GetPlanCost(property.CopDoubleReadTaskType, costFlag) + childCost, err := child.GetPlanCost(property.CopDoubleReadTaskType, option) if err != nil { return 0, err } @@ -204,7 +208,7 @@ func (p *PhysicalIndexLookUpReader) GetPlanCost(taskType property.TaskType, cost } ts := tmp.(*PhysicalTableScan) if p.ctx.GetSessionVars().CostModelVersion == modelVer1 { - tblCost, err := ts.GetPlanCost(property.CopDoubleReadTaskType, costFlag) + tblCost, err := ts.GetPlanCost(property.CopDoubleReadTaskType, option) if err != nil { return 0, err } @@ -257,12 +261,13 @@ func (p *PhysicalIndexLookUpReader) estDoubleReadCost(tbl *model.TableInfo, cost } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalIndexReader) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalIndexReader) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } // child's cost - childCost, err := p.indexPlan.GetPlanCost(property.CopSingleReadTaskType, costFlag) + childCost, err := p.indexPlan.GetPlanCost(property.CopSingleReadTaskType, option) if err != nil { return 0, err } @@ -288,7 +293,8 @@ func (p *PhysicalIndexReader) GetNetDataSize() float64 { } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalTableReader) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalTableReader) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } @@ -297,7 +303,7 @@ func (p *PhysicalTableReader) GetPlanCost(taskType property.TaskType, costFlag u switch p.StoreType { case kv.TiKV: // child's cost - childCost, err := p.tablePlan.GetPlanCost(property.CopSingleReadTaskType, costFlag) + childCost, err := p.tablePlan.GetPlanCost(property.CopSingleReadTaskType, option) if err != nil { return 0, err } @@ -317,7 +323,7 @@ func (p *PhysicalTableReader) GetPlanCost(taskType property.TaskType, costFlag u concurrency = p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor rowSize = collectRowSizeFromMPPPlan(p.tablePlan) seekCost = accumulateNetSeekCost4MPP(p.tablePlan) - childCost, err := p.tablePlan.GetPlanCost(property.MppTaskType, costFlag) + childCost, err := p.tablePlan.GetPlanCost(property.MppTaskType, option) if err != nil { return 0, err } @@ -332,7 +338,7 @@ func (p *PhysicalTableReader) GetPlanCost(taskType property.TaskType, costFlag u // regard the underlying tasks as cop-task on modelVer1 for compatibility tType = property.CopSingleReadTaskType } - childCost, err := p.tablePlan.GetPlanCost(tType, costFlag) + childCost, err := p.tablePlan.GetPlanCost(tType, option) if err != nil { return 0, err } @@ -362,13 +368,14 @@ func (p *PhysicalTableReader) GetNetDataSize() float64 { } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalIndexMergeReader) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalIndexMergeReader) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } p.planCost = 0 if tblScan := p.tablePlan; tblScan != nil { - childCost, err := tblScan.GetPlanCost(property.CopSingleReadTaskType, costFlag) + childCost, err := tblScan.GetPlanCost(property.CopSingleReadTaskType, option) if err != nil { return 0, err } @@ -379,7 +386,7 @@ func (p *PhysicalIndexMergeReader) GetPlanCost(taskType property.TaskType, costF p.planCost += getCardinality(tblScan, costFlag) * rowSize * netFactor // net I/O cost } for _, partialScan := range p.partialPlans { - childCost, err := partialScan.GetPlanCost(property.CopSingleReadTaskType, costFlag) + childCost, err := partialScan.GetPlanCost(property.CopSingleReadTaskType, option) if err != nil { return 0, err } @@ -414,7 +421,8 @@ func (p *PhysicalIndexMergeReader) GetPartialReaderNetDataSize(plan PhysicalPlan } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalTableScan) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalTableScan) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } @@ -451,7 +459,8 @@ func (p *PhysicalTableScan) GetPlanCost(taskType property.TaskType, costFlag uin } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalIndexScan) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalIndexScan) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } @@ -548,16 +557,17 @@ func (p *PhysicalIndexJoin) estDoubleReadCost(doubleReadRows float64) float64 { } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalIndexJoin) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalIndexJoin) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } outerChild, innerChild := p.children[1-p.InnerChildIdx], p.children[p.InnerChildIdx] - outerCost, err := outerChild.GetPlanCost(taskType, costFlag) + outerCost, err := outerChild.GetPlanCost(taskType, option) if err != nil { return 0, err } - innerCost, err := innerChild.GetPlanCost(taskType, costFlag) + innerCost, err := innerChild.GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -636,16 +646,17 @@ func (p *PhysicalIndexHashJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalIndexHashJoin) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalIndexHashJoin) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } outerChild, innerChild := p.children[1-p.InnerChildIdx], p.children[p.InnerChildIdx] - outerCost, err := outerChild.GetPlanCost(taskType, costFlag) + outerCost, err := outerChild.GetPlanCost(taskType, option) if err != nil { return 0, err } - innerCost, err := innerChild.GetPlanCost(taskType, costFlag) + innerCost, err := innerChild.GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -726,16 +737,17 @@ func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCos } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalIndexMergeJoin) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalIndexMergeJoin) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } outerChild, innerChild := p.children[1-p.InnerChildIdx], p.children[p.InnerChildIdx] - outerCost, err := outerChild.GetPlanCost(taskType, costFlag) + outerCost, err := outerChild.GetPlanCost(taskType, option) if err != nil { return 0, err } - innerCost, err := innerChild.GetPlanCost(taskType, costFlag) + innerCost, err := innerChild.GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -778,16 +790,17 @@ func (p *PhysicalApply) GetCost(lCount, rCount, lCost, rCost float64) float64 { } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalApply) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalApply) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } outerChild, innerChild := p.children[1-p.InnerChildIdx], p.children[p.InnerChildIdx] - outerCost, err := outerChild.GetPlanCost(taskType, costFlag) + outerCost, err := outerChild.GetPlanCost(taskType, option) if err != nil { return 0, err } - innerCost, err := innerChild.GetPlanCost(taskType, costFlag) + innerCost, err := innerChild.GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -850,13 +863,14 @@ func (p *PhysicalMergeJoin) GetCost(lCnt, rCnt float64, costFlag uint64) float64 } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalMergeJoin) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalMergeJoin) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } p.planCost = 0 for _, child := range p.children { - childCost, err := child.GetPlanCost(taskType, costFlag) + childCost, err := child.GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -954,13 +968,14 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalHashJoin) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalHashJoin) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } p.planCost = 0 for _, child := range p.children { - childCost, err := child.GetPlanCost(taskType, costFlag) + childCost, err := child.GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -994,11 +1009,12 @@ func (p *PhysicalStreamAgg) GetCost(inputRows float64, isRoot, isMPP bool, costF } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalStreamAgg) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalStreamAgg) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } - childCost, err := p.children[0].GetPlanCost(taskType, costFlag) + childCost, err := p.children[0].GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -1041,11 +1057,12 @@ func (p *PhysicalHashAgg) GetCost(inputRows float64, isRoot, isMPP bool, costFla } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalHashAgg) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalHashAgg) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } - childCost, err := p.children[0].GetPlanCost(taskType, costFlag) + childCost, err := p.children[0].GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -1088,11 +1105,12 @@ func (p *PhysicalSort) GetCost(count float64, schema *expression.Schema) float64 } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalSort) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalSort) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } - childCost, err := p.children[0].GetPlanCost(taskType, costFlag) + childCost, err := p.children[0].GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -1126,11 +1144,12 @@ func (p *PhysicalTopN) GetCost(count float64, isRoot bool) float64 { } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalTopN) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalTopN) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } - childCost, err := p.children[0].GetPlanCost(taskType, costFlag) + childCost, err := p.children[0].GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -1163,7 +1182,8 @@ func (p *BatchPointGetPlan) GetCost() float64 { } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *BatchPointGetPlan) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *BatchPointGetPlan) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } @@ -1185,7 +1205,7 @@ func (p *BatchPointGetPlan) GetAvgRowSize() float64 { } // GetCost returns cost of the PointGetPlan. -func (p *PointGetPlan) GetCost() float64 { +func (p *PointGetPlan) GetCost(opt *physicalOptimizeOp) float64 { cols := p.accessCols if cols == nil { return 0 // the cost of PointGet generated in fast plan optimization is always 0 @@ -1198,18 +1218,22 @@ func (p *PointGetPlan) GetCost() float64 { } else { rowSize = p.stats.HistColl.GetIndexAvgRowSize(p.ctx, cols, p.IndexInfo.Unique) } - cost += rowSize * sessVars.GetNetworkFactor(p.TblInfo) - cost += sessVars.GetSeekFactor(p.TblInfo) + networkFactor := sessVars.GetNetworkFactor(p.TblInfo) + seekFactor := sessVars.GetSeekFactor(p.TblInfo) + cost += rowSize * networkFactor + cost += seekFactor cost /= float64(sessVars.DistSQLScanConcurrency()) + setPointGetPlanCostDetail(p, opt, rowSize, networkFactor, seekFactor) return cost } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PointGetPlan) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PointGetPlan) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } - p.planCost = p.GetCost() + p.planCost = p.GetCost(option.tracer) p.planCostInit = true return p.planCost, nil } @@ -1227,13 +1251,14 @@ func (p *PointGetPlan) GetAvgRowSize() float64 { } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalUnionAll) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalUnionAll) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } var childMaxCost float64 for _, child := range p.children { - childCost, err := child.GetPlanCost(taskType, costFlag) + childCost, err := child.GetPlanCost(taskType, option) if err != nil { return 0, err } @@ -1245,11 +1270,12 @@ func (p *PhysicalUnionAll) GetPlanCost(taskType property.TaskType, costFlag uint } // GetPlanCost calculates the cost of the plan if it has not been calculated yet and returns the cost. -func (p *PhysicalExchangeReceiver) GetPlanCost(taskType property.TaskType, costFlag uint64) (float64, error) { +func (p *PhysicalExchangeReceiver) GetPlanCost(taskType property.TaskType, option *PlanCostOption) (float64, error) { + costFlag := option.CostFlag if p.planCostInit && !hasCostFlag(costFlag, CostFlagRecalculate) { return p.planCost, nil } - childCost, err := p.children[0].GetPlanCost(taskType, costFlag) + childCost, err := p.children[0].GetPlanCost(taskType, option) if err != nil { return 0, err } diff --git a/planner/core/plan_cost_detail.go b/planner/core/plan_cost_detail.go new file mode 100644 index 0000000000000..788dea7984397 --- /dev/null +++ b/planner/core/plan_cost_detail.go @@ -0,0 +1,35 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import "fmt" + +const ( + rowSizeLbl = "rowSize" + networkFactorLbl = "networkFactor" + seekFactorLbl = "seekFactor" +) + +func setPointGetPlanCostDetail(p *PointGetPlan, opt *physicalOptimizeOp, + rowSize, networkFactor, seekFactor float64) { + if opt == nil { + return + } + opt.appendPlanCostDetail(p). + appendParamCostDetail(rowSizeLbl, rowSize). + appendParamCostDetail(networkFactorLbl, networkFactor). + appendParamCostDetail(seekFactorLbl, seekFactor). + appendCurrCostDesc(fmt.Sprintf("%s * %s + %s", rowSizeLbl, networkFactorLbl, seekFactorLbl)) +} diff --git a/util/tracing/opt_trace.go b/util/tracing/opt_trace.go index 09de10828d8a8..98fa5c0a69aaa 100644 --- a/util/tracing/opt_trace.go +++ b/util/tracing/opt_trace.go @@ -157,6 +157,7 @@ func DedupCETrace(records []*CETraceRecord) []*CETraceRecord { // PhysicalOptimizeTracer indicates the trace for the whole physicalOptimize processing type PhysicalOptimizeTracer struct { + PhysicalPlanCostDetails map[int]*PhysicalPlanCostDetail `json:"costs"` // final indicates the final physical plan trace Final []*PlanTrace `json:"final"` Candidates map[int]*CandidatePlanTrace `json:"candidates"` @@ -228,3 +229,36 @@ func (tracer *OptimizeTracer) SetFastPlan(final *PlanTrace) { func (tracer *OptimizeTracer) RecordFinalPlan(final *PlanTrace) { tracer.FinalPlan = toFlattenPlanTrace(final) } + +// PhysicalPlanCostDetail indicates cost detail +type PhysicalPlanCostDetail struct { + id int + tp string + params map[string]interface{} + desc string +} + +// NewPhysicalPlanCostDetail creates a cost detail +func NewPhysicalPlanCostDetail(id int, tp string) *PhysicalPlanCostDetail { + return &PhysicalPlanCostDetail{ + id: id, + tp: tp, + params: make(map[string]interface{}), + } +} + +// AddParam adds param +func (d *PhysicalPlanCostDetail) AddParam(k string, v interface{}) *PhysicalPlanCostDetail { + d.params[k] = v + return d +} + +// SetDesc sets desc +func (d *PhysicalPlanCostDetail) SetDesc(desc string) { + d.desc = desc +} + +// GetPlanID gets plan id +func (d *PhysicalPlanCostDetail) GetPlanID() int { + return d.id +} From e0a83ebcdf7a43c81f7f0f172d4b62d83dd88458 Mon Sep 17 00:00:00 2001 From: yisaer Date: Fri, 29 Jul 2022 13:17:51 +0800 Subject: [PATCH 2/6] add test Signed-off-by: yisaer --- planner/core/find_best_task.go | 27 +++++----- planner/core/plan_cost_detail.go | 14 +++--- planner/core/plan_cost_detail_test.go | 72 +++++++++++++++++++++++++++ util/tracing/opt_trace.go | 11 ++++ 4 files changed, 104 insertions(+), 20 deletions(-) create mode 100644 planner/core/plan_cost_detail_test.go diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 79e86b1f89d24..063209ca0e533 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -204,7 +204,8 @@ func (p *baseLogicalPlan) rebuildChildTasks(childTasks *[]task, pp PhysicalPlan, return nil } -func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty, addEnforcer bool, planCounter *PlanCounterTp, opt *physicalOptimizeOp) (task, int64, error) { +func (p *baseLogicalPlan) 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)) @@ -278,7 +279,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } opt.appendCandidate(p, curTask.plan(), prop) // Get the most efficient one. - if curIsBetter, err := compareTaskCost(p.ctx, curTask, bestTask); err != nil { + if curIsBetter, err := compareTaskCost(p.ctx, curTask, bestTask, opt); err != nil { return nil, 0, err } else if curIsBetter { bestTask = curTask @@ -288,12 +289,12 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } // 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) (curIsBetter bool, err error) { - curCost, curInvalid, err := getTaskPlanCost(curTask) +func compareTaskCost(_ sessionctx.Context, curTask, bestTask task, op *physicalOptimizeOp) (curIsBetter bool, err error) { + curCost, curInvalid, err := getTaskPlanCost(curTask, op) if err != nil { return false, err } - bestCost, bestInvalid, err := getTaskPlanCost(bestTask) + bestCost, bestInvalid, err := getTaskPlanCost(bestTask, op) if err != nil { return false, err } @@ -309,7 +310,7 @@ func compareTaskCost(_ sessionctx.Context, curTask, bestTask task) (curIsBetter // getTaskPlanCost returns the cost of this task. // The new cost interface will be used if EnableNewCostInterface is true. // The second returned value indicates whether this task is valid. -func getTaskPlanCost(t task) (float64, bool, error) { +func getTaskPlanCost(t task, op *physicalOptimizeOp) (float64, bool, error) { if t.invalid() { return math.MaxFloat64, true, nil } @@ -329,7 +330,7 @@ func getTaskPlanCost(t task) (float64, bool, error) { default: return 0, false, errors.New("unknown task type") } - cost, err := t.plan().GetPlanCost(taskType, NewDefaultPlanCostOption()) + cost, err := t.plan().GetPlanCost(taskType, NewDefaultPlanCostOption().WithOptimizeTracer(op)) return cost, false, err } @@ -479,7 +480,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun goto END } opt.appendCandidate(p, curTask.plan(), prop) - if curIsBetter, err := compareTaskCost(p.ctx, curTask, bestTask); err != nil { + if curIsBetter, err := compareTaskCost(p.ctx, curTask, bestTask, opt); err != nil { return nil, 0, err } else if curIsBetter { bestTask = curTask @@ -911,7 +912,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter } appendCandidate(ds, idxMergeTask, prop, opt) - curIsBetter, err := compareTaskCost(ds.ctx, idxMergeTask, t) + curIsBetter, err := compareTaskCost(ds.ctx, idxMergeTask, t, opt) if err != nil { return nil, 0, err } @@ -1003,7 +1004,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter cntPlan++ planCounter.Dec(1) } - curIsBetter, cerr := compareTaskCost(ds.ctx, pointGetTask, t) + curIsBetter, cerr := compareTaskCost(ds.ctx, pointGetTask, t, opt) if cerr != nil { return nil, 0, cerr } @@ -1037,7 +1038,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter planCounter.Dec(1) } appendCandidate(ds, tblTask, prop, opt) - curIsBetter, err := compareTaskCost(ds.ctx, tblTask, t) + curIsBetter, err := compareTaskCost(ds.ctx, tblTask, t, opt) if err != nil { return nil, 0, err } @@ -1062,7 +1063,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter planCounter.Dec(1) } appendCandidate(ds, idxTask, prop, opt) - curIsBetter, err := compareTaskCost(ds.ctx, idxTask, t) + curIsBetter, err := compareTaskCost(ds.ctx, idxTask, t, opt) if err != nil { return nil, 0, err } @@ -2002,7 +2003,7 @@ func (ds *DataSource) convertToSampleTable(prop *property.PhysicalProperty, }, nil } -func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candidate *candidatePath, _ *physicalOptimizeOp) (task task) { +func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candidate *candidatePath, opt *physicalOptimizeOp) (task task) { if !prop.IsSortItemEmpty() && !candidate.isMatchProp { return invalidTask } diff --git a/planner/core/plan_cost_detail.go b/planner/core/plan_cost_detail.go index 788dea7984397..f06e398d8d02a 100644 --- a/planner/core/plan_cost_detail.go +++ b/planner/core/plan_cost_detail.go @@ -17,9 +17,9 @@ package core import "fmt" const ( - rowSizeLbl = "rowSize" - networkFactorLbl = "networkFactor" - seekFactorLbl = "seekFactor" + RowSizeLbl = "rowSize" + NetworkFactorLbl = "networkFactor" + SeekFactorLbl = "seekFactor" ) func setPointGetPlanCostDetail(p *PointGetPlan, opt *physicalOptimizeOp, @@ -28,8 +28,8 @@ func setPointGetPlanCostDetail(p *PointGetPlan, opt *physicalOptimizeOp, return } opt.appendPlanCostDetail(p). - appendParamCostDetail(rowSizeLbl, rowSize). - appendParamCostDetail(networkFactorLbl, networkFactor). - appendParamCostDetail(seekFactorLbl, seekFactor). - appendCurrCostDesc(fmt.Sprintf("%s * %s + %s", rowSizeLbl, networkFactorLbl, seekFactorLbl)) + appendParamCostDetail(RowSizeLbl, rowSize). + appendParamCostDetail(NetworkFactorLbl, networkFactor). + appendParamCostDetail(SeekFactorLbl, seekFactor). + appendCurrCostDesc(fmt.Sprintf("%s * %s + %s", RowSizeLbl, NetworkFactorLbl, SeekFactorLbl)) } diff --git a/planner/core/plan_cost_detail_test.go b/planner/core/plan_cost_detail_test.go new file mode 100644 index 0000000000000..e37b5b675ba6c --- /dev/null +++ b/planner/core/plan_cost_detail_test.go @@ -0,0 +1,72 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core_test + +import ( + "context" + "testing" + + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/hint" + "github.com/pingcap/tidb/util/plancodec" + "github.com/pingcap/tidb/util/tracing" + "github.com/stretchr/testify/require" +) + +func TestPlanCostDetail(t *testing.T) { + p := parser.New() + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t (a int primary key, b int, c int, d int, k int, key b(b), key cd(c, d), unique key(k))`) + // assert PointGet cost detail + testPointGetCostDetail(t, tk, p, dom) +} + +func testPointGetCostDetail(t *testing.T, tk *testkit.TestKit, p *parser.Parser, dom *domain.Domain) { + tk.Session().GetSessionVars().StmtCtx.EnableOptimizeTrace = true + costDetails := optimize(t, "select * from t where a = 1", p, tk.Session(), dom) + assertPG := false + for _, cd := range costDetails { + if cd.GetPlanType() == plancodec.TypePointGet { + assertPG = true + cd.Exists(core.RowSizeLbl) + cd.Exists(core.NetworkFactorLbl) + cd.Exists(core.SeekFactorLbl) + } + } + require.True(t, assertPG) +} + +func optimize(t *testing.T, sql string, p *parser.Parser, ctx sessionctx.Context, dom *domain.Domain) map[int]*tracing.PhysicalPlanCostDetail { + stmt, err := p.ParseOneStmt(sql, "", "") + require.NoError(t, err) + err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: dom.InfoSchema()})) + require.NoError(t, err) + sctx := core.MockContext() + sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true + sctx.GetSessionVars().EnableNewCostInterface = true + builder, _ := core.NewPlanBuilder().Init(sctx, dom.InfoSchema(), &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(dom.InfoSchema()) + plan, err := builder.Build(context.TODO(), stmt) + require.NoError(t, err) + _, _, err = core.DoOptimize(context.TODO(), sctx, builder.GetOptFlag(), plan.(core.LogicalPlan)) + return sctx.GetSessionVars().StmtCtx.OptimizeTracer.Physical.PhysicalPlanCostDetails +} diff --git a/util/tracing/opt_trace.go b/util/tracing/opt_trace.go index 98fa5c0a69aaa..7520c5cc0d4b8 100644 --- a/util/tracing/opt_trace.go +++ b/util/tracing/opt_trace.go @@ -262,3 +262,14 @@ func (d *PhysicalPlanCostDetail) SetDesc(desc string) { func (d *PhysicalPlanCostDetail) GetPlanID() int { return d.id } + +// GetPlanType gets plan type +func (d *PhysicalPlanCostDetail) GetPlanType() string { + return d.tp +} + +// Exists checks whether key exists in params +func (d *PhysicalPlanCostDetail) Exists(k string) bool { + _, ok := d.params[k] + return ok +} From d1271b19feb8d647d358d215c3db849eb4f89b0b Mon Sep 17 00:00:00 2001 From: yisaer Date: Fri, 29 Jul 2022 13:28:15 +0800 Subject: [PATCH 3/6] fix lint Signed-off-by: yisaer --- planner/core/plan_cost_detail.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/planner/core/plan_cost_detail.go b/planner/core/plan_cost_detail.go index f06e398d8d02a..ba5427f346c6d 100644 --- a/planner/core/plan_cost_detail.go +++ b/planner/core/plan_cost_detail.go @@ -17,9 +17,12 @@ package core import "fmt" const ( - RowSizeLbl = "rowSize" + // RowSizeLbl indicates for rowSize + RowSizeLbl = "rowSize" + // NetworkFactorLbl indicates for networkFactor NetworkFactorLbl = "networkFactor" - SeekFactorLbl = "seekFactor" + // SeekFactorLbl indicates for seekFactor + SeekFactorLbl = "seekFactor" ) func setPointGetPlanCostDetail(p *PointGetPlan, opt *physicalOptimizeOp, From 9f5feb06e95928c98c5352f2406e5ea0eda5911d Mon Sep 17 00:00:00 2001 From: yisaer Date: Fri, 29 Jul 2022 15:33:44 +0800 Subject: [PATCH 4/6] fix lint Signed-off-by: yisaer --- planner/core/plan_cost_detail_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/planner/core/plan_cost_detail_test.go b/planner/core/plan_cost_detail_test.go index e37b5b675ba6c..1639da19f8810 100644 --- a/planner/core/plan_cost_detail_test.go +++ b/planner/core/plan_cost_detail_test.go @@ -68,5 +68,6 @@ func optimize(t *testing.T, sql string, p *parser.Parser, ctx sessionctx.Context plan, err := builder.Build(context.TODO(), stmt) require.NoError(t, err) _, _, err = core.DoOptimize(context.TODO(), sctx, builder.GetOptFlag(), plan.(core.LogicalPlan)) + require.NoError(t, err) return sctx.GetSessionVars().StmtCtx.OptimizeTracer.Physical.PhysicalPlanCostDetails } From b5c5746b29e7850385caa0b5057873f4b5aa0add Mon Sep 17 00:00:00 2001 From: yisaer Date: Wed, 3 Aug 2022 11:27:48 +0800 Subject: [PATCH 5/6] address the comment Signed-off-by: yisaer --- planner/core/find_best_task.go | 29 +++------------------------ planner/core/plan_cost_detail.go | 23 ++++++++++++--------- planner/core/plan_cost_detail_test.go | 6 +++--- 3 files changed, 20 insertions(+), 38 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 063209ca0e533..8e42befc74d67 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -337,8 +337,6 @@ func getTaskPlanCost(t task, op *physicalOptimizeOp) (float64, bool, error) { type physicalOptimizeOp struct { // tracer is goring to track optimize steps during physical optimizing tracer *tracing.PhysicalOptimizeTracer - - currCostDetail *tracing.PhysicalPlanCostDetail } func defaultPhysicalOptimizeOption() *physicalOptimizeOp { @@ -362,32 +360,11 @@ func (op *physicalOptimizeOp) appendCandidate(lp LogicalPlan, pp PhysicalPlan, p pp.appendChildCandidate(op) } -func (op *physicalOptimizeOp) appendPlanCostDetail(pp PhysicalPlan) *physicalOptimizeOp { +func (op *physicalOptimizeOp) appendPlanCostDetail(detail *tracing.PhysicalPlanCostDetail) { if op == nil || op.tracer == nil { - return nil - } - if op.currCostDetail != nil { - op.tracer.PhysicalPlanCostDetails[pp.ID()] = op.currCostDetail - op.currCostDetail = nil - } - op.currCostDetail = tracing.NewPhysicalPlanCostDetail(pp.ID(), pp.TP()) - return op -} - -func (op *physicalOptimizeOp) appendParamCostDetail(k string, v interface{}) *physicalOptimizeOp { - if op == nil || op.tracer == nil || op.currCostDetail == nil { - return nil - } - op.currCostDetail.AddParam(k, v) - return op -} - -func (op *physicalOptimizeOp) appendCurrCostDesc(desc string) *physicalOptimizeOp { - if op == nil || op.tracer == nil || op.currCostDetail == nil { - return nil + return } - op.currCostDetail.SetDesc(desc) - return op + op.tracer.PhysicalPlanCostDetails[detail.GetPlanID()] = detail } // findBestTask implements LogicalPlan interface. diff --git a/planner/core/plan_cost_detail.go b/planner/core/plan_cost_detail.go index ba5427f346c6d..ba5f2fe20b196 100644 --- a/planner/core/plan_cost_detail.go +++ b/planner/core/plan_cost_detail.go @@ -14,14 +14,18 @@ package core -import "fmt" +import ( + "fmt" + + "github.com/pingcap/tidb/util/tracing" +) const ( - // RowSizeLbl indicates for rowSize + // RowSizeLbl indicates rowSize RowSizeLbl = "rowSize" - // NetworkFactorLbl indicates for networkFactor + // NetworkFactorLbl indicates networkFactor NetworkFactorLbl = "networkFactor" - // SeekFactorLbl indicates for seekFactor + // SeekFactorLbl indicates seekFactor SeekFactorLbl = "seekFactor" ) @@ -30,9 +34,10 @@ func setPointGetPlanCostDetail(p *PointGetPlan, opt *physicalOptimizeOp, if opt == nil { return } - opt.appendPlanCostDetail(p). - appendParamCostDetail(RowSizeLbl, rowSize). - appendParamCostDetail(NetworkFactorLbl, networkFactor). - appendParamCostDetail(SeekFactorLbl, seekFactor). - appendCurrCostDesc(fmt.Sprintf("%s * %s + %s", RowSizeLbl, NetworkFactorLbl, SeekFactorLbl)) + detail := tracing.NewPhysicalPlanCostDetail(p.ID(), p.TP()) + detail.AddParam(RowSizeLbl, rowSize). + AddParam(NetworkFactorLbl, networkFactor). + AddParam(SeekFactorLbl, seekFactor). + SetDesc(fmt.Sprintf("%s*%s+%s", RowSizeLbl, NetworkFactorLbl, SeekFactorLbl)) + opt.appendPlanCostDetail(detail) } diff --git a/planner/core/plan_cost_detail_test.go b/planner/core/plan_cost_detail_test.go index 1639da19f8810..07d3a76d4eda1 100644 --- a/planner/core/plan_cost_detail_test.go +++ b/planner/core/plan_cost_detail_test.go @@ -47,9 +47,9 @@ func testPointGetCostDetail(t *testing.T, tk *testkit.TestKit, p *parser.Parser, for _, cd := range costDetails { if cd.GetPlanType() == plancodec.TypePointGet { assertPG = true - cd.Exists(core.RowSizeLbl) - cd.Exists(core.NetworkFactorLbl) - cd.Exists(core.SeekFactorLbl) + require.True(t, cd.Exists(core.RowSizeLbl)) + require.True(t, cd.Exists(core.NetworkFactorLbl)) + require.True(t, cd.Exists(core.SeekFactorLbl)) } } require.True(t, assertPG) From a48fbf0d57ce9916556c792ccd84d87c6dfa7d3b Mon Sep 17 00:00:00 2001 From: yisaer Date: Wed, 3 Aug 2022 17:52:08 +0800 Subject: [PATCH 6/6] address the comment Signed-off-by: yisaer --- planner/core/plan_cost_detail_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/planner/core/plan_cost_detail_test.go b/planner/core/plan_cost_detail_test.go index 07d3a76d4eda1..3802ff1efa42f 100644 --- a/planner/core/plan_cost_detail_test.go +++ b/planner/core/plan_cost_detail_test.go @@ -31,8 +31,7 @@ import ( func TestPlanCostDetail(t *testing.T) { p := parser.New() - store, dom, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() + store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table t (a int primary key, b int, c int, d int, k int, key b(b), key cd(c, d), unique key(k))`)