From e0da196b414667a80d8432a7ad433b2219e71f03 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Mon, 22 Aug 2022 15:48:20 +0800 Subject: [PATCH] planner: support HashJoin cost detail (#37012) ref pingcap/tidb#36962 --- planner/core/plan_cost.go | 25 ++- planner/core/plan_cost_detail.go | 215 ++++++++++++++++++++++++++ planner/core/plan_cost_detail_test.go | 74 +++++---- planner/core/task.go | 6 +- planner/implementation/join.go | 2 +- util/tracing/opt_trace.go | 28 ++-- 6 files changed, 296 insertions(+), 54 deletions(-) diff --git a/planner/core/plan_cost.go b/planner/core/plan_cost.go index a2afe704fc23e..93d5fc88a659a 100644 --- a/planner/core/plan_cost.go +++ b/planner/core/plan_cost.go @@ -928,7 +928,7 @@ func (p *PhysicalMergeJoin) GetPlanCost(taskType property.TaskType, option *Plan } // GetCost computes cost of hash join operator itself. -func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint64) float64 { +func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint64, op *physicalOptimizeOp) float64 { buildCnt, probeCnt := lCnt, rCnt build := p.children[0] // Taking the right as the inner for right join or using the outer to build a hash table. @@ -946,9 +946,13 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint if isMPP && p.ctx.GetSessionVars().CostModelVersion == modelVer2 { cpuFactor = sessVars.GetTiFlashCPUFactor() // use the dedicated TiFlash CPU Factor on modelVer2 } + diskFactor := sessVars.GetDiskFactor() + memoryFactor := sessVars.GetMemoryFactor() + concurrencyFactor := sessVars.GetConcurrencyFactor() + cpuCost := buildCnt * cpuFactor - memoryCost := buildCnt * sessVars.GetMemoryFactor() - diskCost := buildCnt * sessVars.GetDiskFactor() * rowSize + memoryCost := buildCnt * memoryFactor + diskCost := buildCnt * diskFactor * rowSize // Number of matched row pairs regarding the equal join conditions. helper := &fullJoinRowCountHelper{ cartesian: false, @@ -982,7 +986,7 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint // Cost of querying hash table is cheap actually, so we just compute the cost of // evaluating `OtherConditions` and joining row pairs. probeCost := numPairs * cpuFactor - probeDiskCost := numPairs * sessVars.GetDiskFactor() * rowSize + probeDiskCost := numPairs * diskFactor * rowSize // Cost of evaluating outer filter. if len(p.LeftConditions)+len(p.RightConditions) > 0 { // Input outer count for the above compution should be adjusted by SelectionFactor. @@ -993,7 +997,7 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint diskCost += probeDiskCost probeCost /= float64(p.Concurrency) // Cost of additional concurrent goroutines. - cpuCost += probeCost + float64(p.Concurrency+1)*sessVars.GetConcurrencyFactor() + cpuCost += probeCost + float64(p.Concurrency+1)*concurrencyFactor // Cost of traveling the hash table to resolve missing matched cases when building the hash table from the outer table if p.UseOuterToBuild { if spill { @@ -1002,7 +1006,7 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint } else { cpuCost += buildCnt * cpuFactor / float64(p.Concurrency) } - diskCost += buildCnt * sessVars.GetDiskFactor() * rowSize + diskCost += buildCnt * diskFactor * rowSize } if spill { @@ -1010,6 +1014,12 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint } else { diskCost = 0 } + if op != nil { + setPhysicalHashJoinCostDetail(p, op, spill, buildCnt, probeCnt, cpuFactor, rowSize, numPairs, + cpuCost, probeCost, memoryCost, diskCost, probeDiskCost, + diskFactor, memoryFactor, concurrencyFactor, + memQuota) + } return cpuCost + memoryCost + diskCost } @@ -1027,7 +1037,8 @@ func (p *PhysicalHashJoin) GetPlanCost(taskType property.TaskType, option *PlanC } p.planCost += childCost } - p.planCost += p.GetCost(getCardinality(p.children[0], costFlag), getCardinality(p.children[1], costFlag), taskType == property.MppTaskType, costFlag) + p.planCost += p.GetCost(getCardinality(p.children[0], costFlag), getCardinality(p.children[1], costFlag), + taskType == property.MppTaskType, costFlag, option.tracer) p.planCostInit = true return p.planCost, nil } diff --git a/planner/core/plan_cost_detail.go b/planner/core/plan_cost_detail.go index 6dab5cf9cc21a..08c701d7ad3a3 100644 --- a/planner/core/plan_cost_detail.go +++ b/planner/core/plan_cost_detail.go @@ -26,6 +26,12 @@ const ( RowCountLbl = "rowCount" // RowSizeLbl indicates rowSize RowSizeLbl = "rowSize" + // BuildRowCountLbl indicates rowCount on build side + BuildRowCountLbl = "buildRowCount" + // ProbeRowCountLbl indicates rowCount on probe side + ProbeRowCountLbl = "probeRowCount" + // NumPairsLbl indicates numPairs + NumPairsLbl = "numPairs" // NetworkFactorLbl indicates networkFactor NetworkFactorLbl = "networkFactor" @@ -33,9 +39,21 @@ const ( SeekFactorLbl = "seekFactor" // ScanFactorLbl indicates for scanFactor ScanFactorLbl = "scanFactor" + // SelectionFactorLbl indicates selection factor + SelectionFactorLbl = "selectionFactor" + // CPUFactorLbl indicates cpu factor + CPUFactorLbl = "cpuFactor" + // MemoryFactorLbl indicates mem factor + MemoryFactorLbl = "memoryFactor" + // DiskFactorLbl indicates disk factor + DiskFactorLbl = "diskFactor" + // ConcurrencyFactorLbl indicates for concurrency factor + ConcurrencyFactorLbl = "concurrencyFactor" // ScanConcurrencyLbl indicates sql scan concurrency ScanConcurrencyLbl = "scanConcurrency" + // HashJoinConcurrencyLbl indicates concurrency for hash join + HashJoinConcurrencyLbl = "hashJoinConcurrency" // NetSeekCostLbl indicates netSeek cost NetSeekCostLbl = "netSeekCost" @@ -43,6 +61,30 @@ const ( TablePlanCostLbl = "tablePlanCost" // IndexPlanCostLbl indicates indexPlan cost IndexPlanCostLbl = "indexPlanCost" + + // ProbeCostDetailLbl indicates probeCost + ProbeCostDetailLbl = "probeCostDetail" + // ProbeCostDescLbl indicates description for probe cost + ProbeCostDescLbl = "probeCostDesc" + // CPUCostDetailLbl indicates cpuCost detail + CPUCostDetailLbl = "cpuCostDetail" + // CPUCostDescLbl indicates description for cpu cost + CPUCostDescLbl = "cpuCostDesc" + // MemCostDetailLbl indicates mem cost detail + MemCostDetailLbl = "memCostDetail" + // MemCostDescLbl indicates description for mem cost + MemCostDescLbl = "memCostDesc" + // DiskCostDetailLbl indicates disk cost detail + DiskCostDetailLbl = "diskCostDetail" + // DiskCostDescLbl indicates description for disk cost + DiskCostDescLbl = "diskCostDesc" + // ProbeDiskCostLbl indicates probe disk cost detail + ProbeDiskCostLbl = "probeDiskCostDetail" + // ProbeDiskCostDescLbl indicates description for probe disk cost + ProbeDiskCostDescLbl = "probeDiskCostDesc" + + // MemQuotaLbl indicates memory quota + MemQuotaLbl = "memQuota" ) func setPointGetPlanCostDetail(p *PointGetPlan, opt *physicalOptimizeOp, @@ -134,3 +176,176 @@ func setPhysicalIndexReaderCostDetail(p *PhysicalIndexReader, opt *physicalOptim RowCountLbl, RowSizeLbl, NetworkFactorLbl, NetSeekCostLbl, ScanConcurrencyLbl)) opt.appendPlanCostDetail(detail) } + +func setPhysicalHashJoinCostDetail(p *PhysicalHashJoin, opt *physicalOptimizeOp, spill bool, + buildCnt, probeCnt, cpuFactor, rowSize, numPairs, + cpuCost, probeCPUCost, memCost, diskCost, probeDiskCost, + diskFactor, memoryFactor, concurrencyFactor float64, + memQuota int64) { + if opt == nil { + return + } + detail := tracing.NewPhysicalPlanCostDetail(p.ID(), p.TP()) + diskCostDetail := &HashJoinDiskCostDetail{ + Spill: spill, + UseOuterToBuild: p.UseOuterToBuild, + BuildRowCount: buildCnt, + DiskFactor: diskFactor, + RowSize: rowSize, + ProbeDiskCost: &HashJoinProbeDiskCostDetail{ + SelectionFactor: SelectionFactor, + NumPairs: numPairs, + HasConditions: len(p.LeftConditions)+len(p.RightConditions) > 0, + Cost: probeDiskCost, + }, + Cost: diskCost, + } + memoryCostDetail := &HashJoinMemoryCostDetail{ + Spill: spill, + MemQuota: memQuota, + RowSize: rowSize, + BuildRowCount: buildCnt, + MemoryFactor: memoryFactor, + Cost: memCost, + } + cpuCostDetail := &HashJoinCPUCostDetail{ + BuildRowCount: buildCnt, + CPUFactor: cpuFactor, + ConcurrencyFactor: concurrencyFactor, + ProbeCost: &HashJoinProbeCostDetail{ + NumPairs: numPairs, + HasConditions: len(p.LeftConditions)+len(p.RightConditions) > 0, + SelectionFactor: SelectionFactor, + ProbeRowCount: probeCnt, + Cost: probeCPUCost, + }, + HashJoinConcurrency: p.Concurrency, + Spill: spill, + Cost: cpuCost, + UseOuterToBuild: p.UseOuterToBuild, + } + + // record cpu cost detail + detail.AddParam(CPUCostDetailLbl, cpuCostDetail). + AddParam(CPUCostDescLbl, cpuCostDetail.desc()). + AddParam(ProbeCostDescLbl, cpuCostDetail.probeCostDesc()) + // record memory cost detail + detail.AddParam(MemCostDetailLbl, memoryCostDetail). + AddParam(MemCostDescLbl, memoryCostDetail.desc()) + // record disk cost detail + detail.AddParam(DiskCostDetailLbl, diskCostDetail). + AddParam(DiskCostDescLbl, diskCostDetail.desc()). + AddParam(ProbeDiskCostDescLbl, diskCostDetail.probeDesc()) + + detail.SetDesc(fmt.Sprintf("%s+%s+%s+all children cost", CPUCostDetailLbl, MemCostDetailLbl, DiskCostDetailLbl)) + opt.appendPlanCostDetail(detail) +} + +// HashJoinProbeCostDetail indicates probe cpu cost detail +type HashJoinProbeCostDetail struct { + NumPairs float64 `json:"numPairs"` + HasConditions bool `json:"hasConditions"` + SelectionFactor float64 `json:"selectionFactor"` + ProbeRowCount float64 `json:"probeRowCount"` + Cost float64 `json:"cost"` +} + +// HashJoinCPUCostDetail indicates cpu cost detail +type HashJoinCPUCostDetail struct { + BuildRowCount float64 `json:"buildRowCount"` + CPUFactor float64 `json:"cpuFactor"` + ConcurrencyFactor float64 `json:"concurrencyFactor"` + ProbeCost *HashJoinProbeCostDetail `json:"probeCost"` + HashJoinConcurrency uint `json:"hashJoinConcurrency"` + Spill bool `json:"spill"` + Cost float64 `json:"cost"` + UseOuterToBuild bool `json:"useOuterToBuild"` +} + +func (h *HashJoinCPUCostDetail) desc() string { + var cpuCostDesc string + buildCostDesc := fmt.Sprintf("%s*%s", BuildRowCountLbl, CPUFactorLbl) + cpuCostDesc = fmt.Sprintf("%s+%s+(%s+1)*%s)", buildCostDesc, ProbeCostDetailLbl, HashJoinConcurrencyLbl, ConcurrencyFactorLbl) + if h.UseOuterToBuild { + if h.Spill { + cpuCostDesc = fmt.Sprintf("%s+%s", cpuCostDesc, buildCostDesc) + } else { + cpuCostDesc = fmt.Sprintf("%s+%s/%s", cpuCostDesc, buildCostDesc, HashJoinConcurrencyLbl) + } + } + return cpuCostDesc +} + +func (h *HashJoinCPUCostDetail) probeCostDesc() string { + var probeCostDesc string + if h.ProbeCost.HasConditions { + probeCostDesc = fmt.Sprintf("(%s*%s*%s+%s*%s)/%s", + NumPairsLbl, CPUFactorLbl, SelectionFactorLbl, + ProbeRowCountLbl, CPUFactorLbl, HashJoinConcurrencyLbl) + } else { + probeCostDesc = fmt.Sprintf("(%s*%s)/%s", + NumPairsLbl, CPUFactorLbl, + HashJoinConcurrencyLbl) + } + return probeCostDesc +} + +// HashJoinMemoryCostDetail indicates memory cost detail +type HashJoinMemoryCostDetail struct { + Spill bool `json:"spill"` + MemQuota int64 `json:"memQuota"` + RowSize float64 `json:"rowSize"` + BuildRowCount float64 `json:"buildRowCount"` + MemoryFactor float64 `json:"memoryFactor"` + Cost float64 `json:"cost"` +} + +func (h *HashJoinMemoryCostDetail) desc() string { + memCostDesc := fmt.Sprintf("%s*%s", BuildRowCountLbl, MemoryFactorLbl) + if h.Spill { + memCostDesc = fmt.Sprintf("%s*%s/(%s*%s)", memCostDesc, MemQuotaLbl, RowSizeLbl, BuildRowCountLbl) + } + return memCostDesc +} + +// HashJoinProbeDiskCostDetail indicates probe disk cost detail +type HashJoinProbeDiskCostDetail struct { + SelectionFactor float64 `json:"selectionFactor"` + NumPairs float64 `json:"numPairs"` + HasConditions bool `json:"hasConditions"` + Cost float64 `json:"cost"` +} + +// HashJoinDiskCostDetail indicates disk cost detail +type HashJoinDiskCostDetail struct { + Spill bool `json:"spill"` + UseOuterToBuild bool `json:"useOuterToBuild"` + BuildRowCount float64 `json:"buildRowCount"` + DiskFactor float64 `json:"diskFactor"` + RowSize float64 `json:"rowSize"` + ProbeDiskCost *HashJoinProbeDiskCostDetail `json:"probeDiskCost"` + Cost float64 `json:"cost"` +} + +func (h *HashJoinDiskCostDetail) desc() string { + if !h.Spill { + return "" + } + buildDiskCost := fmt.Sprintf("%s*%s*%s", BuildRowCountLbl, DiskFactorLbl, RowSizeLbl) + desc := fmt.Sprintf("%s+%s", buildDiskCost, ProbeDiskCostLbl) + if h.UseOuterToBuild { + desc = fmt.Sprintf("%s+%s", desc, buildDiskCost) + } + return desc +} + +func (h *HashJoinDiskCostDetail) probeDesc() string { + if !h.Spill { + return "" + } + desc := fmt.Sprintf("%s*%s*%s", NumPairsLbl, DiskFactorLbl, RowSizeLbl) + if h.ProbeDiskCost.HasConditions { + desc = fmt.Sprintf("%s*%s", desc, SelectionFactorLbl) + } + return desc +} diff --git a/planner/core/plan_cost_detail_test.go b/planner/core/plan_cost_detail_test.go index 8f1fadae88781..34584773aa6e8 100644 --- a/planner/core/plan_cost_detail_test.go +++ b/planner/core/plan_cost_detail_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/planner/core" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/hint" @@ -40,66 +40,78 @@ func TestPlanCostDetail(t *testing.T) { assertLbls []string tp string }{ + { + tp: plancodec.TypeHashJoin, + sql: "select /*+ HASH_JOIN(t1, t2) */ * from t t1 join t t2 on t1.k = t2.k where t1.a = 1;", + assertLbls: []string{ + plannercore.CPUCostDetailLbl, + plannercore.CPUCostDescLbl, + plannercore.ProbeCostDescLbl, + plannercore.MemCostDetailLbl, + plannercore.MemCostDescLbl, + plannercore.DiskCostDetailLbl, + }, + }, { tp: plancodec.TypePointGet, sql: "select * from t where a = 1", assertLbls: []string{ - core.RowSizeLbl, - core.NetworkFactorLbl, - core.SeekFactorLbl, + plannercore.RowSizeLbl, + plannercore.NetworkFactorLbl, + plannercore.SeekFactorLbl, }, }, { tp: plancodec.TypeBatchPointGet, sql: "select * from t where a = 1 or a = 2 or a = 3", assertLbls: []string{ - core.RowCountLbl, - core.RowSizeLbl, - core.NetworkFactorLbl, - core.SeekFactorLbl, - core.ScanConcurrencyLbl, + plannercore.RowCountLbl, + plannercore.RowSizeLbl, + plannercore.NetworkFactorLbl, + plannercore.SeekFactorLbl, + plannercore.ScanConcurrencyLbl, }, }, { tp: plancodec.TypeTableFullScan, sql: "select * from t", assertLbls: []string{ - core.RowCountLbl, - core.RowSizeLbl, - core.ScanFactorLbl, + plannercore.RowCountLbl, + plannercore.RowSizeLbl, + plannercore.ScanFactorLbl, }, }, { tp: plancodec.TypeTableReader, sql: "select * from t", assertLbls: []string{ - core.RowCountLbl, - core.RowSizeLbl, - core.NetworkFactorLbl, - core.NetSeekCostLbl, - core.TablePlanCostLbl, - core.ScanConcurrencyLbl, + plannercore.RowCountLbl, + plannercore.RowSizeLbl, + plannercore.NetworkFactorLbl, + plannercore.NetSeekCostLbl, + plannercore.TablePlanCostLbl, + plannercore.ScanConcurrencyLbl, }, }, { tp: plancodec.TypeIndexFullScan, sql: "select b from t", assertLbls: []string{ - core.RowCountLbl, - core.RowSizeLbl, - core.ScanFactorLbl, + plannercore.RowCountLbl, + plannercore.RowSizeLbl, + plannercore.ScanFactorLbl, }, }, { tp: plancodec.TypeIndexReader, sql: "select b from t", assertLbls: []string{ - core.RowCountLbl, - core.RowSizeLbl, - core.NetworkFactorLbl, - core.NetSeekCostLbl, - core.IndexPlanCostLbl, - core.ScanConcurrencyLbl, + plannercore.RowCountLbl, + plannercore.RowSizeLbl, + plannercore.NetworkFactorLbl, + plannercore.NetSeekCostLbl, + plannercore.IndexPlanCostLbl, + plannercore.ScanConcurrencyLbl, }, }, } @@ -121,16 +133,16 @@ func TestPlanCostDetail(t *testing.T) { 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()})) + err = plannercore.Preprocess(ctx, stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: dom.InfoSchema()})) require.NoError(t, err) - sctx := core.MockContext() + sctx := plannercore.MockContext() sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true sctx.GetSessionVars().EnableNewCostInterface = true - builder, _ := core.NewPlanBuilder().Init(sctx, dom.InfoSchema(), &hint.BlockHintProcessor{}) + builder, _ := plannercore.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)) + _, _, err = plannercore.DoOptimize(context.TODO(), sctx, builder.GetOptFlag(), plan.(plannercore.LogicalPlan)) require.NoError(t, err) return sctx.GetSessionVars().StmtCtx.OptimizeTracer.Physical.PhysicalPlanCostDetails } diff --git a/planner/core/task.go b/planner/core/task.go index 132af680b8be0..da23f255df563 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -329,7 +329,7 @@ func (p *PhysicalHashJoin) attach2Task(tasks ...task) task { p.SetChildren(lTask.plan(), rTask.plan()) task := &rootTask{ p: p, - cst: lTask.cost() + rTask.cost() + p.GetCost(lTask.count(), rTask.count(), false, 0), + cst: lTask.cost() + rTask.cost() + p.GetCost(lTask.count(), rTask.count(), false, 0, nil), } p.cost = task.cost() return task @@ -550,7 +550,7 @@ func (p *PhysicalHashJoin) attach2TaskForMpp(tasks ...task) task { outerTask = rTask } task := &mppTask{ - cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count(), false, 0), + cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count(), false, 0, nil), p: p, partTp: outerTask.partTp, hashCols: outerTask.hashCols, @@ -581,7 +581,7 @@ func (p *PhysicalHashJoin) attach2TaskForTiFlash(tasks ...task) task { tblColHists: rTask.tblColHists, indexPlanFinished: true, tablePlan: p, - cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count(), false, 0), + cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count(), false, 0, nil), } p.cost = task.cst return task diff --git a/planner/implementation/join.go b/planner/implementation/join.go index d601632ec25c1..3c514a26d8a47 100644 --- a/planner/implementation/join.go +++ b/planner/implementation/join.go @@ -29,7 +29,7 @@ func (impl *HashJoinImpl) CalcCost(_ float64, children ...memo.Implementation) f hashJoin := impl.plan.(*plannercore.PhysicalHashJoin) // The children here are only used to calculate the cost. hashJoin.SetChildren(children[0].GetPlan(), children[1].GetPlan()) - selfCost := hashJoin.GetCost(children[0].GetPlan().StatsCount(), children[1].GetPlan().StatsCount(), false, 0) + selfCost := hashJoin.GetCost(children[0].GetPlan().StatsCount(), children[1].GetPlan().StatsCount(), false, 0, nil) impl.cost = selfCost + children[0].GetCost() + children[1].GetCost() return impl.cost } diff --git a/util/tracing/opt_trace.go b/util/tracing/opt_trace.go index 7520c5cc0d4b8..8112379534b46 100644 --- a/util/tracing/opt_trace.go +++ b/util/tracing/opt_trace.go @@ -232,44 +232,48 @@ func (tracer *OptimizeTracer) RecordFinalPlan(final *PlanTrace) { // PhysicalPlanCostDetail indicates cost detail type PhysicalPlanCostDetail struct { - id int - tp string - params map[string]interface{} - desc string + ID int `json:"id"` + TP string `json:"type"` + Params map[string]interface{} `json:"params"` + Desc string `json:"desc"` } // NewPhysicalPlanCostDetail creates a cost detail func NewPhysicalPlanCostDetail(id int, tp string) *PhysicalPlanCostDetail { return &PhysicalPlanCostDetail{ - id: id, - tp: tp, - params: make(map[string]interface{}), + 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 + // discard empty param value + if s, ok := v.(string); ok && len(s) < 1 { + return d + } + d.Params[k] = v return d } // SetDesc sets desc func (d *PhysicalPlanCostDetail) SetDesc(desc string) { - d.desc = desc + d.Desc = desc } // GetPlanID gets plan id func (d *PhysicalPlanCostDetail) GetPlanID() int { - return d.id + return d.ID } // GetPlanType gets plan type func (d *PhysicalPlanCostDetail) GetPlanType() string { - return d.tp + return d.TP } // Exists checks whether key exists in params func (d *PhysicalPlanCostDetail) Exists(k string) bool { - _, ok := d.params[k] + _, ok := d.Params[k] return ok }