From f0a744704bee8454411190d5099cf541af07c86a Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Thu, 9 May 2024 19:38:38 +0800 Subject: [PATCH] planner: move cost factors to core/cost pkg (#53120) ref pingcap/tidb#51664, ref pingcap/tidb#52714 --- pkg/planner/core/BUILD.bazel | 1 + pkg/planner/core/cost/BUILD.bazel | 9 ++++ pkg/planner/core/cost/factors_thresholds.go | 54 +++++++++++++++++++++ pkg/planner/core/exhaust_physical_plans.go | 7 +-- pkg/planner/core/find_best_task.go | 41 +++------------- pkg/planner/core/indexmerge_path.go | 5 +- pkg/planner/core/logical_plans.go | 9 ++-- pkg/planner/core/physical_plans.go | 7 +-- pkg/planner/core/plan_cost_detail.go | 5 +- pkg/planner/core/plan_cost_ver1.go | 45 ++++++++--------- pkg/planner/core/stats.go | 9 ++-- pkg/planner/core/task.go | 3 +- pkg/planner/core/task_base.go | 3 +- pkg/planner/implementation/BUILD.bazel | 1 + pkg/planner/implementation/simple_plans.go | 3 +- 15 files changed, 124 insertions(+), 78 deletions(-) create mode 100644 pkg/planner/core/cost/BUILD.bazel create mode 100644 pkg/planner/core/cost/factors_thresholds.go diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index bff990ffd6d9e..8138a1feb7a64 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -112,6 +112,7 @@ go_library( "//pkg/planner/cardinality", "//pkg/planner/context", "//pkg/planner/core/base", + "//pkg/planner/core/cost", "//pkg/planner/core/metrics", "//pkg/planner/core/operator/baseimpl", "//pkg/planner/funcdep", diff --git a/pkg/planner/core/cost/BUILD.bazel b/pkg/planner/core/cost/BUILD.bazel new file mode 100644 index 0000000000000..05ee2b26fb415 --- /dev/null +++ b/pkg/planner/core/cost/BUILD.bazel @@ -0,0 +1,9 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "cost", + srcs = ["factors_thresholds.go"], + importpath = "github.com/pingcap/tidb/pkg/planner/core/cost", + visibility = ["//visibility:public"], + deps = ["//pkg/parser/ast"], +) diff --git a/pkg/planner/core/cost/factors_thresholds.go b/pkg/planner/core/cost/factors_thresholds.go new file mode 100644 index 0000000000000..cdd101babd956 --- /dev/null +++ b/pkg/planner/core/cost/factors_thresholds.go @@ -0,0 +1,54 @@ +// Copyright 2024 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 cost + +import "github.com/pingcap/tidb/pkg/parser/ast" + +// ******************************* Factors ******************************* +const ( + // SelectionFactor is the default factor of the selectivity. + // For example, If we have no idea how to estimate the selectivity + // of a Selection or a JoinCondition, we can use this default value. + SelectionFactor = 0.8 + + DistinctFactor = 0.8 +) + +// AggFuncFactor is the basic factor for aggregation. +var AggFuncFactor = map[string]float64{ + ast.AggFuncCount: 1.0, + ast.AggFuncSum: 1.0, + ast.AggFuncAvg: 2.0, + ast.AggFuncFirstRow: 0.1, + ast.AggFuncMax: 1.0, + ast.AggFuncMin: 1.0, + ast.AggFuncGroupConcat: 1.0, + ast.AggFuncBitOr: 0.9, + ast.AggFuncBitXor: 0.9, + ast.AggFuncBitAnd: 0.9, + ast.AggFuncVarPop: 3.0, + ast.AggFuncVarSamp: 3.0, + ast.AggFuncStddevPop: 3.0, + ast.AggFuncStddevSamp: 3.0, + "default": 1.5, +} + +// // ******************************* Thresholds ******************************* +const ( + // SmallScanThreshold means: If the actual row count is much more + // than the limit count, the unordered scan may cost much more than keep order. + // So when a limit exists, we don't apply the DescScanFactor. + SmallScanThreshold = 10000 +) diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index c0474a7273014..27cbb51d6b56b 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" @@ -1078,7 +1079,7 @@ func (p *LogicalJoin) constructInnerTableScanTask( selectivity, _, err = cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, ts.filterCondition, ds.possibleAccessPaths) if err != nil || selectivity <= 0 { logutil.BgLogger().Debug("unexpected selectivity, use selection factor", zap.Float64("selectivity", selectivity), zap.String("table", ts.TableAsName.L)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } // rowCount is computed from result row count of join, which has already accounted the filters on DataSource, // i.e, rowCount equals to `countAfterAccess * selectivity`. @@ -1370,7 +1371,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, tblConds, ds.possibleAccessPaths) if err != nil || selectivity <= 0 { logutil.BgLogger().Debug("unexpected selectivity, use selection factor", zap.Float64("selectivity", selectivity), zap.String("table", ds.TableAsName.L)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } // rowCount is computed from result row count of join, which has already accounted the filters on DataSource, // i.e, rowCount equals to `countAfterIndex * selectivity`. @@ -1388,7 +1389,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, indexConds, ds.possibleAccessPaths) if err != nil || selectivity <= 0 { logutil.BgLogger().Debug("unexpected selectivity, use selection factor", zap.Float64("selectivity", selectivity), zap.String("table", ds.TableAsName.L)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } cnt := tmpPath.CountAfterIndex / selectivity if rowCountUpperBound > 0 { diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 6279d6468c33f..636e6c664edb2 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" @@ -47,36 +48,6 @@ import ( "go.uber.org/zap" ) -const ( - // SelectionFactor is the default factor of the selectivity. - // For example, If we have no idea how to estimate the selectivity - // of a Selection or a JoinCondition, we can use this default value. - SelectionFactor = 0.8 - distinctFactor = 0.8 - - // If the actual row count is much more than the limit count, the unordered scan may cost much more than keep order. - // So when a limit exists, we don't apply the DescScanFactor. - smallScanThreshold = 10000 -) - -var aggFuncFactor = map[string]float64{ - ast.AggFuncCount: 1.0, - ast.AggFuncSum: 1.0, - ast.AggFuncAvg: 2.0, - ast.AggFuncFirstRow: 0.1, - ast.AggFuncMax: 1.0, - ast.AggFuncMin: 1.0, - ast.AggFuncGroupConcat: 1.0, - ast.AggFuncBitOr: 0.9, - ast.AggFuncBitXor: 0.9, - ast.AggFuncBitAnd: 0.9, - ast.AggFuncVarPop: 3.0, - ast.AggFuncVarSamp: 3.0, - ast.AggFuncStddevPop: 3.0, - ast.AggFuncStddevSamp: 3.0, - "default": 1.5, -} - // PlanCounterDisabled is the default value of PlanCounterTp, indicating that optimizer needn't force a plan. var PlanCounterDisabled base.PlanCounterTp = -1 @@ -1042,7 +1013,7 @@ func (ds *DataSource) matchPropForIndexMergeAlternatives(path *util.AccessPath, sel, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, []expression.Expression{accessDNF}, nil) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) - sel = SelectionFactor + sel = cost.SelectionFactor } indexMergePath.CountAfterAccess = sel * ds.tableStats.RowCount if noSortItem { @@ -1730,7 +1701,7 @@ func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, ts.filterCondition, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } tablePlan = PhysicalSelection{Conditions: ts.filterCondition}.Init(ts.SCtx(), ts.StatsInfo().ScaleByExpectCnt(selectivity*rowCount), ds.QueryBlockOffset()) tablePlan.SetChildren(ts) @@ -1816,7 +1787,7 @@ func (ds *DataSource) buildIndexMergeTableScan(tableFilters []expression.Express selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, pushedFilters, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } sel := PhysicalSelection{Conditions: pushedFilters}.Init(ts.SCtx(), ts.StatsInfo().ScaleByExpectCnt(selectivity*totalRowCount), ts.QueryBlockOffset()) sel.SetChildren(ts) @@ -2299,7 +2270,7 @@ func (is *PhysicalIndexScan) addPushedDownSelection(copTask *CopTask, p *DataSou selectivity, _, err := cardinality.Selectivity(is.SCtx(), copTask.tblColHists, tableConds, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } tableSel.SetStats(copTask.Plan().StatsInfo().Scale(selectivity)) } @@ -2790,7 +2761,7 @@ func (ts *PhysicalTableScan) addPushedDownSelection(copTask *CopTask, stats *pro selectivity, _, err := cardinality.Selectivity(ts.SCtx(), copTask.tblColHists, ts.filterCondition, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } sel.SetStats(ts.StatsInfo().Scale(selectivity)) } diff --git a/pkg/planner/core/indexmerge_path.go b/pkg/planner/core/indexmerge_path.go index 77c00ee489067..3e5ba1fb98407 100644 --- a/pkg/planner/core/indexmerge_path.go +++ b/pkg/planner/core/indexmerge_path.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/context" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" @@ -286,7 +287,7 @@ func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) sel, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, []expression.Expression{accessDNF}, nil) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) - sel = SelectionFactor + sel = cost.SelectionFactor } possiblePath := buildIndexMergeOrPath(filters, partialAlternativePaths, k, shouldKeepCurrentFilter) @@ -594,7 +595,7 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int, usedAccessMa sel, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, partialFilters, nil) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) - sel = SelectionFactor + sel = cost.SelectionFactor } indexMergePath := &util.AccessPath{ diff --git a/pkg/planner/core/logical_plans.go b/pkg/planner/core/logical_plans.go index daec1701720b5..38528ed97c6b5 100644 --- a/pkg/planner/core/logical_plans.go +++ b/pkg/planner/core/logical_plans.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/cost" fd "github.com/pingcap/tidb/pkg/planner/funcdep" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" @@ -1687,7 +1688,7 @@ func (ds *DataSource) deriveCommonHandleTablePathStats(path *util.AccessPath, co // If the `CountAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. // We prefer the `stats.RowCount` because it could use more stats info to calculate the selectivity. if path.CountAfterAccess < ds.StatsInfo().RowCount && !isIm { - path.CountAfterAccess = math.Min(ds.StatsInfo().RowCount/SelectionFactor, float64(ds.statisticTable.RealtimeCount)) + path.CountAfterAccess = math.Min(ds.StatsInfo().RowCount/cost.SelectionFactor, float64(ds.statisticTable.RealtimeCount)) } return nil } @@ -1773,7 +1774,7 @@ func (ds *DataSource) deriveTablePathStats(path *util.AccessPath, conds []expres // If the `CountAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. // We prefer the `stats.RowCount` because it could use more stats info to calculate the selectivity. if path.CountAfterAccess < ds.StatsInfo().RowCount && !isIm { - path.CountAfterAccess = math.Min(ds.StatsInfo().RowCount/SelectionFactor, float64(ds.statisticTable.RealtimeCount)) + path.CountAfterAccess = math.Min(ds.StatsInfo().RowCount/cost.SelectionFactor, float64(ds.statisticTable.RealtimeCount)) } return err } @@ -1844,13 +1845,13 @@ func (ds *DataSource) deriveIndexPathStats(path *util.AccessPath, _ []expression // If the `CountAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. // We prefer the `stats.RowCount` because it could use more stats info to calculate the selectivity. if path.CountAfterAccess < ds.StatsInfo().RowCount && !isIm { - path.CountAfterAccess = math.Min(ds.StatsInfo().RowCount/SelectionFactor, float64(ds.statisticTable.RealtimeCount)) + path.CountAfterAccess = math.Min(ds.StatsInfo().RowCount/cost.SelectionFactor, float64(ds.statisticTable.RealtimeCount)) } if path.IndexFilters != nil { selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, path.IndexFilters, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } if isIm { path.CountAfterIndex = path.CountAfterAccess * selectivity diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index 56bd7a9370f92..ed043548b9e08 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/coreusage" @@ -1915,10 +1916,10 @@ func (p *basePhysicalAgg) numDistinctFunc() (num int) { func (p *basePhysicalAgg) getAggFuncCostFactor(isMPP bool) (factor float64) { factor = 0.0 for _, agg := range p.AggFuncs { - if fac, ok := aggFuncFactor[agg.Name]; ok { + if fac, ok := cost.AggFuncFactor[agg.Name]; ok { factor += fac } else { - factor += aggFuncFactor["default"] + factor += cost.AggFuncFactor["default"] } } if factor == 0 { @@ -1927,7 +1928,7 @@ func (p *basePhysicalAgg) getAggFuncCostFactor(isMPP bool) (factor float64) { // But in mpp cases, 2-phase is more usual. So we change this factor. // TODO: This is still a little tricky and might cause regression. We should // calibrate these factors and polish our cost model in the future. - factor = aggFuncFactor[ast.AggFuncFirstRow] + factor = cost.AggFuncFactor[ast.AggFuncFirstRow] } else { factor = 1.0 } diff --git a/pkg/planner/core/plan_cost_detail.go b/pkg/planner/core/plan_cost_detail.go index 84e008ff780c2..0efeaf754cec1 100644 --- a/pkg/planner/core/plan_cost_detail.go +++ b/pkg/planner/core/plan_cost_detail.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" "github.com/pingcap/tidb/pkg/util/tracing" ) @@ -195,7 +196,7 @@ func setPhysicalHashJoinCostDetail(p *PhysicalHashJoin, opt *optimizetrace.Physi DiskFactor: diskFactor, RowSize: rowSize, ProbeDiskCost: &HashJoinProbeDiskCostDetail{ - SelectionFactor: SelectionFactor, + SelectionFactor: cost.SelectionFactor, NumPairs: numPairs, HasConditions: len(p.LeftConditions)+len(p.RightConditions) > 0, Cost: probeDiskCost, @@ -217,7 +218,7 @@ func setPhysicalHashJoinCostDetail(p *PhysicalHashJoin, opt *optimizetrace.Physi ProbeCost: &HashJoinProbeCostDetail{ NumPairs: numPairs, HasConditions: len(p.LeftConditions)+len(p.RightConditions) > 0, - SelectionFactor: SelectionFactor, + SelectionFactor: cost.SelectionFactor, ProbeRowCount: probeCnt, Cost: probeCPUCost, }, diff --git a/pkg/planner/core/plan_cost_ver1.go b/pkg/planner/core/plan_cost_ver1.go index af395ce45af28..ea8a1f7cca290 100644 --- a/pkg/planner/core/plan_cost_ver1.go +++ b/pkg/planner/core/plan_cost_ver1.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/costusage" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" @@ -424,7 +425,7 @@ func (p *PhysicalTableScan) GetPlanCostVer1(_ property.TaskType, option *optimiz var rowCount, rowSize, scanFactor float64 costModelVersion := p.SCtx().GetSessionVars().CostModelVersion scanFactor = p.SCtx().GetSessionVars().GetScanFactor(p.Table) - if p.Desc && p.prop != nil && p.prop.ExpectedCnt >= smallScanThreshold { + if p.Desc && p.prop != nil && p.prop.ExpectedCnt >= cost.SmallScanThreshold { scanFactor = p.SCtx().GetSessionVars().GetDescScanFactor(p.Table) } rowCount = getCardinality(p, costFlag) @@ -449,7 +450,7 @@ func (p *PhysicalIndexScan) GetPlanCostVer1(_ property.TaskType, option *optimiz var rowCount, rowSize, scanFactor float64 costModelVersion := p.SCtx().GetSessionVars().CostModelVersion scanFactor = p.SCtx().GetSessionVars().GetScanFactor(p.Table) - if p.Desc && p.prop != nil && p.prop.ExpectedCnt >= smallScanThreshold { + if p.Desc && p.prop != nil && p.prop.ExpectedCnt >= cost.SmallScanThreshold { scanFactor = p.SCtx().GetSessionVars().GetDescScanFactor(p.Table) } rowCount = getCardinality(p, costFlag) @@ -472,7 +473,7 @@ func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost flo // summed length of left/right conditions. if len(p.LeftConditions)+len(p.RightConditions) > 0 { cpuCost += sessVars.GetCPUFactor() * outerCnt - outerCnt *= SelectionFactor + outerCnt *= cost.SelectionFactor } // Cost of extracting lookup keys. innerCPUCost := sessVars.GetCPUFactor() * outerCnt @@ -483,12 +484,12 @@ func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost flo innerCPUCost += outerCnt * (math.Log2(batchSize) + 1) * sessVars.GetCPUFactor() } // Add cost of building inner executors. CPU cost of building copTasks: - // (outerCnt / batchSize) * (batchSize * distinctFactor) * CPUFactor + // (outerCnt / batchSize) * (batchSize * DistinctFactor) * CPUFactor // Since we don't know the number of copTasks built, ignore these network cost now. - innerCPUCost += outerCnt * distinctFactor * sessVars.GetCPUFactor() + innerCPUCost += outerCnt * cost.DistinctFactor * sessVars.GetCPUFactor() // CPU cost of building hash table for inner results: - // (outerCnt / batchSize) * (batchSize * distinctFactor) * innerCnt * CPUFactor - innerCPUCost += outerCnt * distinctFactor * innerCnt * sessVars.GetCPUFactor() + // (outerCnt / batchSize) * (batchSize * DistinctFactor) * innerCnt * CPUFactor + innerCPUCost += outerCnt * cost.DistinctFactor * innerCnt * sessVars.GetCPUFactor() innerConcurrency := float64(p.SCtx().GetSessionVars().IndexLookupJoinConcurrency()) cpuCost += innerCPUCost / innerConcurrency // Cost of probing hash table in main thread. @@ -509,7 +510,7 @@ func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost flo cpuCost += probeCost + (innerConcurrency+1.0)*sessVars.GetConcurrencyFactor() // Memory cost of hash tables for inner rows. The computed result is the upper bound, // since the executor is pipelined and not all workers are always in full load. - memoryCost := innerConcurrency * (batchSize * distinctFactor) * innerCnt * sessVars.GetMemoryFactor() + memoryCost := innerConcurrency * (batchSize * cost.DistinctFactor) * innerCnt * sessVars.GetMemoryFactor() // Cost of inner child plan, i.e, mainly I/O and network cost. innerPlanCost := outerCnt * innerCost if p.SCtx().GetSessionVars().CostModelVersion == 2 { @@ -558,7 +559,7 @@ func (p *PhysicalIndexHashJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost // summed length of left/right conditions. if len(p.LeftConditions)+len(p.RightConditions) > 0 { cpuCost += sessVars.GetCPUFactor() * outerCnt - outerCnt *= SelectionFactor + outerCnt *= cost.SelectionFactor } // Cost of extracting lookup keys. innerCPUCost := sessVars.GetCPUFactor() * outerCnt @@ -569,9 +570,9 @@ func (p *PhysicalIndexHashJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost innerCPUCost += outerCnt * (math.Log2(batchSize) + 1) * sessVars.GetCPUFactor() } // Add cost of building inner executors. CPU cost of building copTasks: - // (outerCnt / batchSize) * (batchSize * distinctFactor) * CPUFactor + // (outerCnt / batchSize) * (batchSize * DistinctFactor) * CPUFactor // Since we don't know the number of copTasks built, ignore these network cost now. - innerCPUCost += outerCnt * distinctFactor * sessVars.GetCPUFactor() + innerCPUCost += outerCnt * cost.DistinctFactor * sessVars.GetCPUFactor() concurrency := float64(sessVars.IndexLookupJoinConcurrency()) cpuCost += innerCPUCost / concurrency // CPU cost of building hash table for outer results concurrently. @@ -606,7 +607,7 @@ func (p *PhysicalIndexHashJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost cpuCost += (concurrency + 1.0) * sessVars.GetConcurrencyFactor() // Memory cost of hash tables for outer rows. The computed result is the upper bound, // since the executor is pipelined and not all workers are always in full load. - memoryCost := concurrency * (batchSize * distinctFactor) * innerCnt * sessVars.GetMemoryFactor() + memoryCost := concurrency * (batchSize * cost.DistinctFactor) * innerCnt * sessVars.GetMemoryFactor() // Cost of inner child plan, i.e, mainly I/O and network cost. innerPlanCost := outerCnt * innerCost return outerCost + innerPlanCost + cpuCost + memoryCost @@ -647,7 +648,7 @@ func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCos // summed length of left/right conditions. if len(p.LeftConditions)+len(p.RightConditions) > 0 { cpuCost += sessVars.GetCPUFactor() * outerCnt - outerCnt *= SelectionFactor + outerCnt *= cost.SelectionFactor } // Cost of extracting lookup keys. innerCPUCost := sessVars.GetCPUFactor() * outerCnt @@ -664,9 +665,9 @@ func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCos innerCPUCost += outerCnt * (sortFactor + 1.0) * sessVars.GetCPUFactor() } // Add cost of building inner executors. CPU cost of building copTasks: - // (outerCnt / batchSize) * (batchSize * distinctFactor) * cpuFactor + // (outerCnt / batchSize) * (batchSize * DistinctFactor) * cpuFactor // Since we don't know the number of copTasks built, ignore these network cost now. - innerCPUCost += outerCnt * distinctFactor * sessVars.GetCPUFactor() + innerCPUCost += outerCnt * cost.DistinctFactor * sessVars.GetCPUFactor() innerConcurrency := float64(p.SCtx().GetSessionVars().IndexLookupJoinConcurrency()) cpuCost += innerCPUCost / innerConcurrency // Cost of merge join in inner worker. @@ -735,11 +736,11 @@ func (p *PhysicalApply) GetCost(lCount, rCount, lCost, rCost float64) float64 { sessVars := p.SCtx().GetSessionVars() if len(p.LeftConditions) > 0 { cpuCost += lCount * sessVars.GetCPUFactor() - lCount *= SelectionFactor + lCount *= cost.SelectionFactor } if len(p.RightConditions) > 0 { cpuCost += lCount * rCount * sessVars.GetCPUFactor() - rCount *= SelectionFactor + rCount *= cost.SelectionFactor } if len(p.EqualConditions)+len(p.OtherConditions)+len(p.NAEqualConditions) > 0 { if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin || @@ -813,7 +814,7 @@ func (p *PhysicalMergeJoin) GetCost(lCnt, rCnt float64, costFlag uint64) float64 // Cost of evaluating outer filters. var cpuCost float64 if len(p.LeftConditions)+len(p.RightConditions) > 0 { - probeCost *= SelectionFactor + probeCost *= cost.SelectionFactor cpuCost += outerCnt * sessVars.GetCPUFactor() } cpuCost += probeCost @@ -898,8 +899,8 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, _ bool, costFlag uint64, // 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. - probeCost *= SelectionFactor - probeDiskCost *= SelectionFactor + probeCost *= cost.SelectionFactor + probeDiskCost *= cost.SelectionFactor probeCost += probeCnt * cpuFactor } diskCost += probeDiskCost @@ -962,7 +963,7 @@ func (p *PhysicalStreamAgg) GetCost(inputRows float64, isRoot, _ bool, costFlag cpuCost = inputRows * sessVars.GetCopCPUFactor() * aggFuncFactor } rowsPerGroup := inputRows / getCardinality(p, costFlag) - memoryCost := rowsPerGroup * distinctFactor * sessVars.GetMemoryFactor() * float64(p.numDistinctFunc()) + memoryCost := rowsPerGroup * cost.DistinctFactor * sessVars.GetMemoryFactor() * float64(p.numDistinctFunc()) return cpuCost + memoryCost } @@ -1003,7 +1004,7 @@ func (p *PhysicalHashAgg) GetCost(inputRows float64, isRoot, isMPP bool, costFla memoryCost := cardinality * sessVars.GetMemoryFactor() * float64(len(p.AggFuncs)) // When aggregation has distinct flag, we would allocate a map for each group to // check duplication. - memoryCost += inputRows * distinctFactor * sessVars.GetMemoryFactor() * float64(numDistinctFunc) + memoryCost += inputRows * cost.DistinctFactor * sessVars.GetMemoryFactor() * float64(numDistinctFunc) return cpuCost + memoryCost } diff --git a/pkg/planner/core/stats.go b/pkg/planner/core/stats.go index 9b3c0dcdc5b69..d8f944f80afa1 100644 --- a/pkg/planner/core/stats.go +++ b/pkg/planner/core/stats.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" @@ -289,7 +290,7 @@ func (ds *DataSource) deriveStatsByFilter(conds expression.CNFExprs, filledPaths selectivity, _, err := cardinality.Selectivity(ds.SCtx(), ds.tableStats.HistColl, conds, filledPaths) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } // TODO: remove NewHistCollBySelectivity later on. // if ds.SCtx().GetSessionVars().OptimizerSelectivityLevel >= 1 { @@ -583,7 +584,7 @@ func (p *LogicalSelection) DeriveStats(childStats []*property.StatsInfo, _ *expr if p.StatsInfo() != nil { return p.StatsInfo(), nil } - p.SetStats(childStats[0].Scale(SelectionFactor)) + p.SetStats(childStats[0].Scale(cost.SelectionFactor)) p.StatsInfo().GroupNDVs = nil return p.StatsInfo(), nil } @@ -815,11 +816,11 @@ func (p *LogicalJoin) DeriveStats(childStats []*property.StatsInfo, selfSchema * nil, nil) if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin { p.SetStats(&property.StatsInfo{ - RowCount: leftProfile.RowCount * SelectionFactor, + RowCount: leftProfile.RowCount * cost.SelectionFactor, ColNDVs: make(map[int64]float64, len(leftProfile.ColNDVs)), }) for id, c := range leftProfile.ColNDVs { - p.StatsInfo().ColNDVs[id] = c * SelectionFactor + p.StatsInfo().ColNDVs[id] = c * cost.SelectionFactor } return p.StatsInfo(), nil } diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index c0dc02a636a4a..87f88d34b22da 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/core/operator/baseimpl" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" @@ -597,7 +598,7 @@ func (t *CopTask) handleRootTaskConds(ctx base.PlanContext, newTask *RootTask) { selectivity, _, err := cardinality.Selectivity(ctx, t.tblColHists, t.rootTaskConds, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } sel := PhysicalSelection{Conditions: t.rootTaskConds}.Init(ctx, newTask.GetPlan().StatsInfo().Scale(selectivity), newTask.GetPlan().QueryBlockOffset()) sel.fromDataSource = true diff --git a/pkg/planner/core/task_base.go b/pkg/planner/core/task_base.go index 1d8f5080427fe..bfc7ef9526d73 100644 --- a/pkg/planner/core/task_base.go +++ b/pkg/planner/core/task_base.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/util/logutil" @@ -205,7 +206,7 @@ func (t *MppTask) ConvertToRootTaskImpl(ctx base.PlanContext) *RootTask { selectivity, _, err := cardinality.Selectivity(ctx, t.tblColHists, t.rootTaskConds, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) - selectivity = SelectionFactor + selectivity = cost.SelectionFactor } sel := PhysicalSelection{Conditions: t.rootTaskConds}.Init(ctx, rt.GetPlan().StatsInfo().Scale(selectivity), rt.GetPlan().QueryBlockOffset()) sel.fromDataSource = true diff --git a/pkg/planner/implementation/BUILD.bazel b/pkg/planner/implementation/BUILD.bazel index 37e1667760c84..d0af00ea0580f 100644 --- a/pkg/planner/implementation/BUILD.bazel +++ b/pkg/planner/implementation/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/planner/cardinality", "//pkg/planner/core", "//pkg/planner/core/base", + "//pkg/planner/core/cost", "//pkg/planner/memo", "//pkg/statistics", ], diff --git a/pkg/planner/implementation/simple_plans.go b/pkg/planner/implementation/simple_plans.go index 283cf151412e9..a5aa170e857e3 100644 --- a/pkg/planner/implementation/simple_plans.go +++ b/pkg/planner/implementation/simple_plans.go @@ -16,6 +16,7 @@ package implementation import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/core/cost" "github.com/pingcap/tidb/pkg/planner/memo" ) @@ -230,7 +231,7 @@ func (impl *ApplyImpl) GetCostLimit(costLimit float64, children ...memo.Implemen leftCount, leftCost := children[0].GetPlan().StatsInfo().RowCount, children[0].GetCost() apply := impl.plan.(*plannercore.PhysicalApply) if len(apply.LeftConditions) > 0 { - leftCount *= plannercore.SelectionFactor + leftCount *= cost.SelectionFactor } return (costLimit - leftCost) / leftCount }