From 108f04d78605985956c22a06792d80112ebd59e2 Mon Sep 17 00:00:00 2001 From: tpp <146148086+terry1purcell@users.noreply.github.com> Date: Mon, 9 Dec 2024 18:16:00 +0800 Subject: [PATCH] planner: Use/force to apply prefer range scan (#56928) close pingcap/tidb#55632 --- .../test/issuetest/executor_issue_test.go | 2 + pkg/planner/core/BUILD.bazel | 1 + .../casetest/dag/testdata/plan_suite_out.json | 6 +- .../testdata/plan_suite_out.json | 8 +- .../testdata/plan_stats_suite_out.json | 9 +- .../testdata/plan_normalized_suite_out.json | 8 +- pkg/planner/core/find_best_task.go | 9 +- pkg/planner/core/logical_plans_test.go | 2 +- pkg/planner/core/plan_cost_ver2.go | 85 +++++++---- pkg/planner/core/plan_cost_ver2_test.go | 46 +++++- pkg/planner/core/stats.go | 27 ++-- pkg/sessionctx/stmtctx/stmtctx.go | 12 ++ .../r/planner/core/plan_cost_ver2.result | 132 +++++++++--------- 13 files changed, 224 insertions(+), 123 deletions(-) diff --git a/pkg/executor/test/issuetest/executor_issue_test.go b/pkg/executor/test/issuetest/executor_issue_test.go index b577aab7e9eec..db90b8028a36c 100644 --- a/pkg/executor/test/issuetest/executor_issue_test.go +++ b/pkg/executor/test/issuetest/executor_issue_test.go @@ -321,12 +321,14 @@ func TestIndexJoin31494(t *testing.T) { insertStr += fmt.Sprintf(", (%d, %d)", i, i) } tk.MustExec(insertStr) + tk.MustExec("analyze table t1") tk.MustExec("create table t2(a int(11) default null, b int(11) default null, c int(11) default null)") insertStr = "insert into t2 values(1, 1, 1)" for i := 1; i < 32768; i++ { insertStr += fmt.Sprintf(", (%d, %d, %d)", i, i, i) } tk.MustExec(insertStr) + tk.MustExec("analyze table t2") sm := &testkit.MockSessionManager{ PS: make([]*util.ProcessInfo, 0), } diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 87c87974d51bc..94444d34019df 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -324,6 +324,7 @@ go_test( "//pkg/util/ranger", "//pkg/util/stmtsummary", "//pkg/util/tracing", + "//tests/realtikvtest", "@com_github_docker_go_units//:go-units", "@com_github_golang_snappy//:snappy", "@com_github_pingcap_errors//:errors", diff --git a/pkg/planner/core/casetest/dag/testdata/plan_suite_out.json b/pkg/planner/core/casetest/dag/testdata/plan_suite_out.json index e1c111054aa02..c173504285016 100644 --- a/pkg/planner/core/casetest/dag/testdata/plan_suite_out.json +++ b/pkg/planner/core/casetest/dag/testdata/plan_suite_out.json @@ -48,7 +48,7 @@ }, { "SQL": "select c from t order by t.a limit 1", - "Best": "IndexReader(Index(t.c_d_e)[[NULL,+inf]]->TopN([test.t.a],0,1))->TopN([test.t.a],0,1)->Projection" + "Best": "TableReader(Table(t)->Limit)->Limit->Projection" }, { "SQL": "select c from t order by t.a + t.b limit 1", @@ -165,7 +165,7 @@ }, { "SQL": "select * from t t1 join t t2 on t1.b = t2.a order by t1.a", - "Best": "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.b,test.t.a)->Sort" + "Best": "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.b,test.t.a)" }, { "SQL": "select * from t t1 join t t2 on t1.b = t2.a order by t1.a limit 1", @@ -508,7 +508,7 @@ }, { "SQL": "select a from t union all (select c from t) order by a limit 1", - "Best": "UnionAll{IndexReader(Index(t.f)[[NULL,+inf]]->TopN([test.t.a],0,1))->TopN([test.t.a],0,1)->IndexReader(Index(t.c_d_e)[[NULL,+inf]]->Limit)->Limit}->TopN([Column#25],0,1)" + "Best": "UnionAll{TableReader(Table(t)->Limit)->Limit->IndexReader(Index(t.c_d_e)[[NULL,+inf]]->Limit)->Limit}->TopN([Column#25],0,1)" } ] }, diff --git a/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json b/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json index 8ab9807f5a9f2..28965bdfc2107 100644 --- a/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json +++ b/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json @@ -2257,11 +2257,11 @@ "Cases": [ { "SQL": "select max(a) from t;", - "Best": "IndexReader(Index(t.f)[[NULL,+inf]]->TopN([test.t.a true],0,1))->TopN([test.t.a true],0,1)->StreamAgg" + "Best": "TableReader(Table(t)->Limit)->Limit->StreamAgg" }, { "SQL": "select min(a) from t;", - "Best": "IndexReader(Index(t.f)[[NULL,+inf]]->TopN([test.t.a],0,1))->TopN([test.t.a],0,1)->StreamAgg" + "Best": "TableReader(Table(t)->Limit)->Limit->StreamAgg" }, { "SQL": "select min(c_str) from t;", @@ -2277,7 +2277,7 @@ }, { "SQL": "select max(a), min(a) from t;", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]]->TopN([test.t.a true],0,1))->TopN([test.t.a true],0,1)->StreamAgg->IndexReader(Index(t.f)[[NULL,+inf]]->TopN([test.t.a],0,1))->TopN([test.t.a],0,1)->StreamAgg}" + "Best": "LeftHashJoin{TableReader(Table(t)->Limit)->Limit->StreamAgg->TableReader(Table(t)->Limit)->Limit->StreamAgg}" }, { "SQL": "select max(a), min(a) from t where a > 10", @@ -2289,7 +2289,7 @@ }, { "SQL": "select max(a), max(c), min(f) from t", - "Best": "LeftHashJoin{LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]]->TopN([test.t.a true],0,1))->TopN([test.t.a true],0,1)->StreamAgg->IndexReader(Index(t.c_d_e)[[NULL,+inf]]->Limit)->Limit->StreamAgg}->IndexReader(Index(t.f)[[NULL,+inf]]->Limit)->Limit->StreamAgg}" + "Best": "LeftHashJoin{LeftHashJoin{TableReader(Table(t)->Limit)->Limit->StreamAgg->IndexReader(Index(t.c_d_e)[[NULL,+inf]]->Limit)->Limit->StreamAgg}->IndexReader(Index(t.f)[[NULL,+inf]]->Limit)->Limit->StreamAgg}" }, { "SQL": "select max(a), max(b) from t", diff --git a/pkg/planner/core/casetest/planstats/testdata/plan_stats_suite_out.json b/pkg/planner/core/casetest/planstats/testdata/plan_stats_suite_out.json index cf49e628aae38..d2d4b06a17e7a 100644 --- a/pkg/planner/core/casetest/planstats/testdata/plan_stats_suite_out.json +++ b/pkg/planner/core/casetest/planstats/testdata/plan_stats_suite_out.json @@ -125,13 +125,14 @@ "Query": "explain format = brief select * from t join tp where tp.a = 10 and t.b = tp.c", "Result": [ "Projection 1.00 root test.t.a, test.t.b, test.t.c, test.tp.a, test.tp.b, test.tp.c", - "└─HashJoin 1.00 root inner join, equal:[eq(test.tp.c, test.t.b)]", + "└─IndexJoin 1.00 root inner join, inner:IndexLookUp, outer key:test.tp.c, inner key:test.t.b, equal cond:eq(test.tp.c, test.t.b)", " ├─TableReader(Build) 1.00 root partition:p1 data:Selection", " │ └─Selection 1.00 cop[tikv] eq(test.tp.a, 10), not(isnull(test.tp.c))", " │ └─TableFullScan 6.00 cop[tikv] table:tp keep order:false, stats:partial[c:allEvicted]", - " └─TableReader(Probe) 3.00 root data:Selection", - " └─Selection 3.00 cop[tikv] not(isnull(test.t.b))", - " └─TableFullScan 3.00 cop[tikv] table:t keep order:false, stats:partial[idx:allEvicted, a:allEvicted, b:allEvicted]" + " └─IndexLookUp(Probe) 1.00 root ", + " ├─Selection(Build) 1.00 cop[tikv] not(isnull(test.t.b))", + " │ └─IndexRangeScan 1.00 cop[tikv] table:t, index:idx(b) range: decided by [eq(test.t.b, test.tp.c)], keep order:false, stats:partial[idx:allEvicted, a:allEvicted, b:allEvicted]", + " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false, stats:partial[idx:allEvicted, a:allEvicted, b:allEvicted]" ] }, { diff --git a/pkg/planner/core/casetest/testdata/plan_normalized_suite_out.json b/pkg/planner/core/casetest/testdata/plan_normalized_suite_out.json index 7be4636f8f444..23a285aa8c495 100644 --- a/pkg/planner/core/casetest/testdata/plan_normalized_suite_out.json +++ b/pkg/planner/core/casetest/testdata/plan_normalized_suite_out.json @@ -428,8 +428,8 @@ "Plan": [ " TableReader root ", " └─ExchangeSender cop[tiflash] ", - " └─Selection cop[tiflash] gt(test.t1.a, ?)", - " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.b, ?), gt(test.t1.c, ?), keep order:false" + " └─Selection cop[tiflash] gt(test.t1.c, ?)", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), gt(test.t1.b, ?), keep order:false" ] }, { @@ -463,8 +463,8 @@ "Plan": [ " TableReader root ", " └─ExchangeSender cop[tiflash] ", - " └─Selection cop[tiflash] gt(test.t1.b, ?), gt(test.t1.c, ?), or(gt(test.t1.a, ?), lt(test.t1.b, ?))", - " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), keep order:false" + " └─Selection cop[tiflash] gt(test.t1.a, ?), gt(test.t1.c, ?), or(gt(test.t1.a, ?), lt(test.t1.b, ?))", + " └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.b, ?), keep order:false" ] }, { diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index ff13b1667923e..97a25d8be261e 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -1172,13 +1172,15 @@ func skylinePruning(ds *logicalop.DataSource, prop *property.PhysicalProperty) [ } } - preferRange := ds.SCtx().GetSessionVars().GetAllowPreferRangeScan() && (ds.TableStats.HistColl.Pseudo || ds.TableStats.RowCount < 1) // If we've forced an index merge - we want to keep these plans preferMerge := len(ds.IndexMergeHints) > 0 || fixcontrol.GetBoolWithDefault( ds.SCtx().GetSessionVars().GetOptimizerFixControlMap(), fixcontrol.Fix52869, false, ) + // tidb_opt_prefer_range_scan is the master switch to control index preferencing + preferRange := ds.SCtx().GetSessionVars().GetAllowPreferRangeScan() && + (preferMerge || (ds.TableStats.HistColl.Pseudo || ds.TableStats.RowCount < 1)) if preferRange && len(candidates) > 1 { // If a candidate path is TiFlash-path or forced-path or MV index, we just keep them. For other candidate paths, if there exists // any range scan path, we remove full scan paths and keep range scan paths. @@ -1197,9 +1199,8 @@ func skylinePruning(ds *logicalop.DataSource, prop *property.PhysicalProperty) [ } if !ranger.HasFullRange(c.path.Ranges, unsignedIntHandle) { // Preference plans with equals/IN predicates or where there is more filtering in the index than against the table - equalPlan := c.path.EqCondCount > 0 || c.path.EqOrInCondCount > 0 - indexFilters := len(c.path.TableFilters) < len(c.path.IndexFilters) - if preferMerge || (((equalPlan || indexFilters) && prop.IsSortItemEmpty()) || c.isMatchProp) { + indexFilters := c.path.EqCondCount > 0 || c.path.EqOrInCondCount > 0 || len(c.path.TableFilters) < len(c.path.IndexFilters) + if preferMerge || (indexFilters && (prop.IsSortItemEmpty() || c.isMatchProp)) { preferredPaths = append(preferredPaths, c) hasRangeScanPath = true } diff --git a/pkg/planner/core/logical_plans_test.go b/pkg/planner/core/logical_plans_test.go index 938bc799aa2a5..c05f5003892a9 100644 --- a/pkg/planner/core/logical_plans_test.go +++ b/pkg/planner/core/logical_plans_test.go @@ -2020,7 +2020,7 @@ func TestSkylinePruning(t *testing.T) { }, { sql: "select * from pt2_global_index where b > 1 order by b", - result: "b_global,b_c_global", + result: "PRIMARY_KEY,b_global,b_c_global", }, { sql: "select b from pt2_global_index where b > 1 order by b", diff --git a/pkg/planner/core/plan_cost_ver2.go b/pkg/planner/core/plan_cost_ver2.go index 8f92cd2146f4e..f57e319d0c811 100644 --- a/pkg/planner/core/plan_cost_ver2.go +++ b/pkg/planner/core/plan_cost_ver2.go @@ -163,30 +163,8 @@ func (p *PhysicalTableScan) GetPlanCostVer2(taskType property.TaskType, option * if p.StoreType == kv.TiFlash { p.PlanCostVer2 = costusage.SumCostVer2(p.PlanCostVer2, scanCostVer2(option, TiFlashStartupRowPenalty, rowSize, scanFactor)) } else if !p.isChildOfIndexLookUp { - // Apply cost penalty for full scans that carry high risk of underestimation - sessionVars := p.SCtx().GetSessionVars() - allowPreferRangeScan := sessionVars.GetAllowPreferRangeScan() - tblColHists := p.tblColHists - - // hasUnreliableStats is a check for pseudo or zero stats - hasUnreliableStats := tblColHists.Pseudo || tblColHists.RealtimeCount < 1 - // hasHighModifyCount tracks the high risk of a tablescan where auto-analyze had not yet updated the table row count - hasHighModifyCount := tblColHists.ModifyCount > tblColHists.RealtimeCount - // hasLowEstimate is a check to capture a unique customer case where modifyCount is used for tablescan estimate (but it not adequately understood why) - hasLowEstimate := rows > 1 && tblColHists.ModifyCount < tblColHists.RealtimeCount && int64(rows) <= tblColHists.ModifyCount - // preferRangeScan check here is same as in skylinePruning - preferRangeScanCondition := allowPreferRangeScan && (hasUnreliableStats || hasHighModifyCount || hasLowEstimate) - var unsignedIntHandle bool - if p.Table.PKIsHandle { - if pkColInfo := p.Table.GetPkColInfo(); pkColInfo != nil { - unsignedIntHandle = mysql.HasUnsignedFlag(pkColInfo.GetFlag()) - } - } - hasFullRangeScan := ranger.HasFullRange(p.Ranges, unsignedIntHandle) - - shouldApplyPenalty := hasFullRangeScan && preferRangeScanCondition - if shouldApplyPenalty { - newRowCount := max(MaxPenaltyRowCount, max(float64(tblColHists.ModifyCount), float64(tblColHists.RealtimeCount))) + newRowCount := getTableScanPenalty(p, rows) + if newRowCount > 0 { p.PlanCostVer2 = costusage.SumCostVer2(p.PlanCostVer2, scanCostVer2(option, newRowCount, rowSize, scanFactor)) } } @@ -935,6 +913,65 @@ func doubleReadCostVer2(option *optimizetrace.PlanCostOption, numTasks float64, func() string { return fmt.Sprintf("doubleRead(tasks(%v)*%v)", numTasks, requestFactor) }) } +func getTableScanPenalty(p *PhysicalTableScan, rows float64) (rowPenalty float64) { + // Apply cost penalty for full scans that carry high risk of underestimation. Exclude those + // that are the child of an index scan or child is TableRangeScan + if len(p.rangeInfo) > 0 { + return float64(0) + } + var unsignedIntHandle bool + if p.Table.PKIsHandle { + if pkColInfo := p.Table.GetPkColInfo(); pkColInfo != nil { + unsignedIntHandle = mysql.HasUnsignedFlag(pkColInfo.GetFlag()) + } + } + hasFullRangeScan := ranger.HasFullRange(p.Ranges, unsignedIntHandle) + if !hasFullRangeScan { + return float64(0) + } + + sessionVars := p.SCtx().GetSessionVars() + allowPreferRangeScan := sessionVars.GetAllowPreferRangeScan() + tblColHists := p.tblColHists + originalRows := int64(tblColHists.GetAnalyzeRowCount()) + + // hasUnreliableStats is a check for pseudo or zero stats + hasUnreliableStats := tblColHists.Pseudo || originalRows < 1 + // hasHighModifyCount tracks the high risk of a tablescan where auto-analyze had not yet updated the table row count + hasHighModifyCount := tblColHists.ModifyCount > originalRows + // hasLowEstimate is a check to capture a unique customer case where modifyCount is used for tablescan estimate (but it not adequately understood why) + hasLowEstimate := rows > 1 && tblColHists.ModifyCount < originalRows && int64(rows) <= tblColHists.ModifyCount + // preferRangeScan check here is same as in skylinePruning + preferRangeScanCondition := allowPreferRangeScan && (hasUnreliableStats || hasHighModifyCount || hasLowEstimate) + + // differentiate a FullTableScan from a partition level scan - so we shouldn't penalize these + hasPartitionScan := false + if p.PlanPartInfo != nil { + if len(p.PlanPartInfo.PruningConds) > 0 { + hasPartitionScan = true + } + } + + // GetIndexForce assumes that the USE/FORCE index is to force a range scan, and thus the + // penalty is applied to a full table scan (not range scan). This may also penalize a + // full table scan where USE/FORCE was applied to the primary key. + hasIndexForce := sessionVars.StmtCtx.GetIndexForce() + shouldApplyPenalty := hasFullRangeScan && (hasIndexForce || preferRangeScanCondition) + if shouldApplyPenalty { + // MySQL will increase the cost of table scan if FORCE index is used. TiDB takes this one + // step further - because we don't differentiate USE/FORCE - the added penalty applies to + // both, and it also applies to any full table scan in the query. Use "max" to get the minimum + // number of rows to add as a penalty to the table scan. + minRows := max(MaxPenaltyRowCount, rows) + if hasPartitionScan { + return minRows + } + // If it isn't a partitioned table - choose the max that includes ModifyCount + return max(minRows, float64(tblColHists.ModifyCount)) + } + return float64(0) +} + // In Cost Ver2, we hide cost factors from users and deprecate SQL variables like `tidb_opt_scan_factor`. type costVer2Factors struct { TiDBTemp costusage.CostVer2Factor // operations on TiDB temporary table diff --git a/pkg/planner/core/plan_cost_ver2_test.go b/pkg/planner/core/plan_cost_ver2_test.go index c78572ffc9c24..614548b27c0aa 100644 --- a/pkg/planner/core/plan_cost_ver2_test.go +++ b/pkg/planner/core/plan_cost_ver2_test.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/tests/realtikvtest" "github.com/stretchr/testify/require" ) @@ -59,9 +60,9 @@ func TestCostModelVer2ScanRowSize(t *testing.T) { {"select a, b from t use index(abc) where a=1 and b=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"}, {"select a, b, c from t use index(abc) where a=1 and b=1 and c=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"}, // table scan row-size is always equal to row-size(*) - {"select a from t use index(primary) where a=1", "(scan(1*logrowsize(80)*tikv_scan_factor(40.7))) + (scan(10000*logrowsize(80)*tikv_scan_factor(40.7)))"}, - {"select a, d from t use index(primary) where a=1", "(scan(1*logrowsize(80)*tikv_scan_factor(40.7))) + (scan(10000*logrowsize(80)*tikv_scan_factor(40.7)))"}, - {"select * from t use index(primary) where a=1", "(scan(1*logrowsize(80)*tikv_scan_factor(40.7))) + (scan(10000*logrowsize(80)*tikv_scan_factor(40.7)))"}, + {"select a from t use index(primary) where a=1", "(scan(1*logrowsize(80)*tikv_scan_factor(40.7))) + (scan(1000*logrowsize(80)*tikv_scan_factor(40.7)))"}, + {"select a, d from t use index(primary) where a=1", "(scan(1*logrowsize(80)*tikv_scan_factor(40.7))) + (scan(1000*logrowsize(80)*tikv_scan_factor(40.7)))"}, + {"select * from t use index(primary) where a=1", "(scan(1*logrowsize(80)*tikv_scan_factor(40.7))) + (scan(1000*logrowsize(80)*tikv_scan_factor(40.7)))"}, } for _, c := range cases { rs := tk.MustQuery("explain analyze format=true_card_cost " + c.query).Rows() @@ -162,3 +163,42 @@ func BenchmarkGetPlanCost(b *testing.B) { _, _ = core.GetPlanCost(phyPlan, property.RootTaskType, optimizetrace.NewDefaultPlanCostOption().WithCostFlag(costusage.CostFlagRecalculate)) } } + +func TestTableScanCostWithForce(t *testing.T) { + store, dom := realtikvtest.CreateMockStoreAndDomainAndSetup(t) + defer func() { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + dom.StatsHandle().Clear() + }() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, primary key (a))") + + // Insert some data + tk.MustExec("insert into t values (1,1),(2,2),(3,3),(4,4),(5,5)") + + // Analyze table to update statistics + tk.MustExec("analyze table t") + + // Test TableFullScan with and without FORCE INDEX + rs := tk.MustQuery("explain analyze format=verbose select * from t").Rows() + planCost1 := rs[0][2].(string) + rs = tk.MustQuery("explain analyze format=verbose select * from t force index(PRIMARY)").Rows() + planCost2 := rs[0][2].(string) + + // Query with FORCE should be more expensive than query without + require.Less(t, planCost1, planCost2) + + // Test TableRangeScan with and without FORCE INDEX + rs = tk.MustQuery("explain analyze format=verbose select * from t where a > 1").Rows() + planCost1 = rs[0][2].(string) + rs = tk.MustQuery("explain analyze format=verbose select * from t force index(PRIMARY) where a > 1").Rows() + planCost2 = rs[0][2].(string) + + // Query costs should be equal since FORCE cost penalty does not apply to range scan + require.Equal(t, planCost1, planCost2) +} diff --git a/pkg/planner/core/stats.go b/pkg/planner/core/stats.go index 40056528f55ee..a0680f4af88ef 100644 --- a/pkg/planner/core/stats.go +++ b/pkg/planner/core/stats.go @@ -160,7 +160,11 @@ func deriveStats4DataSource(lp base.LogicalPlan, colGroups [][]*expression.Colum if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { debugTraceAccessPaths(ds.SCtx(), ds.PossibleAccessPaths) } - ds.AccessPathMinSelectivity = getMinSelectivityFromPaths(ds.PossibleAccessPaths, float64(ds.TblColHists.RealtimeCount)) + indexForce := false + ds.AccessPathMinSelectivity, indexForce = getGeneralAttributesFromPaths(ds.PossibleAccessPaths, float64(ds.TblColHists.RealtimeCount)) + if indexForce { + ds.SCtx().GetSessionVars().StmtCtx.SetIndexForce() + } return ds.StatsInfo(), nil } @@ -404,21 +408,24 @@ func detachCondAndBuildRangeForPath( return err } -func getMinSelectivityFromPaths(paths []*util.AccessPath, totalRowCount float64) float64 { +func getGeneralAttributesFromPaths(paths []*util.AccessPath, totalRowCount float64) (float64, bool) { minSelectivity := 1.0 - if totalRowCount <= 0 { - return minSelectivity - } + indexForce := false for _, path := range paths { // For table path and index merge path, AccessPath.CountAfterIndex is not set and meaningless, // but we still consider their AccessPath.CountAfterAccess. - if path.IsTablePath() || path.PartialIndexPaths != nil { - minSelectivity = min(minSelectivity, path.CountAfterAccess/totalRowCount) - continue + if totalRowCount > 0 { + if path.IsTablePath() || path.PartialIndexPaths != nil { + minSelectivity = min(minSelectivity, path.CountAfterAccess/totalRowCount) + } else { + minSelectivity = min(minSelectivity, path.CountAfterIndex/totalRowCount) + } + } + if !indexForce && path.Forced { + indexForce = true } - minSelectivity = min(minSelectivity, path.CountAfterIndex/totalRowCount) } - return minSelectivity + return minSelectivity, indexForce } func getGroupNDVs(ds *logicalop.DataSource, colGroups [][]*expression.Column) []property.GroupNDV { diff --git a/pkg/sessionctx/stmtctx/stmtctx.go b/pkg/sessionctx/stmtctx/stmtctx.go index 209551719a42d..d80f0eae487d9 100644 --- a/pkg/sessionctx/stmtctx/stmtctx.go +++ b/pkg/sessionctx/stmtctx/stmtctx.go @@ -288,6 +288,8 @@ type StatementContext struct { planHint string planHintSet bool binaryPlan string + // indexForce is set if any table in the query has a force or use index applied + indexForce bool // To avoid cycle import, we use interface{} for the following two fields. // flatPlan should be a *plannercore.FlatPhysicalPlan if it's not nil flatPlan any @@ -732,6 +734,11 @@ func (sc *StatementContext) GetPlanHint() (string, bool) { return sc.planHint, sc.planHintSet } +// GetIndexForce gets the IndexForce boolean generated from the plan. +func (sc *StatementContext) GetIndexForce() bool { + return sc.indexForce +} + // InitDiskTracker initializes the sc.DiskTracker, use cache to avoid allocation. func (sc *StatementContext) InitDiskTracker(label int, bytesLimit int64) { memory.InitTracker(&sc.cache.DiskTracker, label, bytesLimit, &sc.cache.LogOnExceed[0]) @@ -750,6 +757,11 @@ func (sc *StatementContext) SetPlanHint(hint string) { sc.planHint = hint } +// SetIndexForce sets the hint for the plan. +func (sc *StatementContext) SetIndexForce() { + sc.indexForce = true +} + // PlanCacheType is the flag of plan cache type PlanCacheType int diff --git a/tests/integrationtest/r/planner/core/plan_cost_ver2.result b/tests/integrationtest/r/planner/core/plan_cost_ver2.result index 939db6b4b7b24..3d03758d83ec8 100644 --- a/tests/integrationtest/r/planner/core/plan_cost_ver2.result +++ b/tests/integrationtest/r/planner/core/plan_cost_ver2.result @@ -51,16 +51,16 @@ IndexReader_6 100.00 22616.00 root index:IndexRangeScan_5 └─IndexRangeScan_5 100.00 16280.00 cop[tikv] table:t, index:b(b) range:[-inf,100), keep order:false explain format='verbose' select /*+ use_index(t, primary) */ a from t; id estRows estCost task access object operator info -TableReader_5 100.00 29066.80 root data:TableFullScan_4 -└─TableFullScan_4 100.00 22730.80 cop[tikv] table:t keep order:false +TableReader_5 100.00 256374.77 root data:TableFullScan_4 +└─TableFullScan_4 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary) */ a, b from t; id estRows estCost task access object operator info -TableReader_5 100.00 35402.80 root data:TableFullScan_4 -└─TableFullScan_4 100.00 22730.80 cop[tikv] table:t keep order:false +TableReader_5 100.00 262710.77 root data:TableFullScan_4 +└─TableFullScan_4 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary) */ a, b, c from t; id estRows estCost task access object operator info -TableReader_5 100.00 41738.80 root data:TableFullScan_4 -└─TableFullScan_4 100.00 22730.80 cop[tikv] table:t keep order:false +TableReader_5 100.00 269046.77 root data:TableFullScan_4 +└─TableFullScan_4 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, b) */ * from t where b<1; id estRows estCost task access object operator info IndexLookUp_7 1.00 10241.58 root @@ -78,91 +78,91 @@ IndexLookUp_7 100.00 1024158.30 root └─TableRowIDScan_6(Probe) 100.00 22730.80 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary) */ a from t where mod(a, 20)<10; id estRows estCost task access object operator info -TableReader_7 80.00 32789.60 root data:Selection_6 -└─Selection_6 80.00 27720.80 cop[tikv] lt(mod(planner__core__plan_cost_ver2.t.a, 20), 10) - └─TableFullScan_5 100.00 22730.80 cop[tikv] table:t keep order:false +TableReader_7 80.00 260097.57 root data:Selection_6 +└─Selection_6 80.00 255028.77 cop[tikv] lt(mod(planner__core__plan_cost_ver2.t.a, 20), 10) + └─TableFullScan_5 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary) */ a from t where mod(a, 20)<10 and mod(a, 20)<11; id estRows estCost task access object operator info -TableReader_7 80.00 37779.60 root data:Selection_6 -└─Selection_6 80.00 32710.80 cop[tikv] lt(mod(planner__core__plan_cost_ver2.t.a, 20), 10), lt(mod(planner__core__plan_cost_ver2.t.a, 20), 11) - └─TableFullScan_5 100.00 22730.80 cop[tikv] table:t keep order:false +TableReader_7 80.00 265087.57 root data:Selection_6 +└─Selection_6 80.00 260018.77 cop[tikv] lt(mod(planner__core__plan_cost_ver2.t.a, 20), 10), lt(mod(planner__core__plan_cost_ver2.t.a, 20), 11) + └─TableFullScan_5 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary) */ a from t where mod(a, 20)<10 and mod(a, 20)<11 and mod(a, 20)<12; id estRows estCost task access object operator info -TableReader_7 80.00 42769.60 root data:Selection_6 -└─Selection_6 80.00 37700.80 cop[tikv] lt(mod(planner__core__plan_cost_ver2.t.a, 20), 10), lt(mod(planner__core__plan_cost_ver2.t.a, 20), 11), lt(mod(planner__core__plan_cost_ver2.t.a, 20), 12) - └─TableFullScan_5 100.00 22730.80 cop[tikv] table:t keep order:false +TableReader_7 80.00 270077.57 root data:Selection_6 +└─Selection_6 80.00 265008.77 cop[tikv] lt(mod(planner__core__plan_cost_ver2.t.a, 20), 10), lt(mod(planner__core__plan_cost_ver2.t.a, 20), 11), lt(mod(planner__core__plan_cost_ver2.t.a, 20), 12) + └─TableFullScan_5 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary) */ a+1 from t; id estRows estCost task access object operator info -Projection_3 100.00 34056.80 root plus(planner__core__plan_cost_ver2.t.a, 1)->Column#4 -└─TableReader_5 100.00 29066.80 root data:TableFullScan_4 - └─TableFullScan_4 100.00 22730.80 cop[tikv] table:t keep order:false +Projection_3 100.00 261364.77 root plus(planner__core__plan_cost_ver2.t.a, 1)->Column#4 +└─TableReader_5 100.00 256374.77 root data:TableFullScan_4 + └─TableFullScan_4 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary) */ a+1, a+2 from t; id estRows estCost task access object operator info -Projection_3 100.00 39046.80 root plus(planner__core__plan_cost_ver2.t.a, 1)->Column#4, plus(planner__core__plan_cost_ver2.t.a, 2)->Column#5 -└─TableReader_5 100.00 29066.80 root data:TableFullScan_4 - └─TableFullScan_4 100.00 22730.80 cop[tikv] table:t keep order:false +Projection_3 100.00 266354.77 root plus(planner__core__plan_cost_ver2.t.a, 1)->Column#4, plus(planner__core__plan_cost_ver2.t.a, 2)->Column#5 +└─TableReader_5 100.00 256374.77 root data:TableFullScan_4 + └─TableFullScan_4 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary) */ a+1, a+2, a+3 from t; id estRows estCost task access object operator info -Projection_3 100.00 44036.80 root plus(planner__core__plan_cost_ver2.t.a, 1)->Column#4, plus(planner__core__plan_cost_ver2.t.a, 2)->Column#5, plus(planner__core__plan_cost_ver2.t.a, 3)->Column#6 -└─TableReader_5 100.00 29066.80 root data:TableFullScan_4 - └─TableFullScan_4 100.00 22730.80 cop[tikv] table:t keep order:false +Projection_3 100.00 271344.77 root plus(planner__core__plan_cost_ver2.t.a, 1)->Column#4, plus(planner__core__plan_cost_ver2.t.a, 2)->Column#5, plus(planner__core__plan_cost_ver2.t.a, 3)->Column#6 +└─TableReader_5 100.00 256374.77 root data:TableFullScan_4 + └─TableFullScan_4 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary), hash_agg() */ sum(a) from t group by b; id estRows estCost task access object operator info -HashAgg_7 100.00 62699.70 root group by:Column#7, funcs:sum(Column#6)->Column#4 -└─Projection_13 100.00 40442.70 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#6, planner__core__plan_cost_ver2.t.b->Column#7 - └─TableReader_12 100.00 35402.80 root data:TableFullScan_11 - └─TableFullScan_11 100.00 22730.80 cop[tikv] table:t keep order:false +HashAgg_7 100.00 290007.67 root group by:Column#7, funcs:sum(Column#6)->Column#4 +└─Projection_13 100.00 267750.67 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#6, planner__core__plan_cost_ver2.t.b->Column#7 + └─TableReader_12 100.00 262710.77 root data:TableFullScan_11 + └─TableFullScan_11 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary), hash_agg() */ sum(a), sum(a+2) from t group by b; id estRows estCost task access object operator info -HashAgg_7 100.00 68489.70 root group by:Column#10, funcs:sum(Column#8)->Column#4, funcs:sum(Column#9)->Column#5 -└─Projection_13 100.00 45432.70 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#8, cast(plus(planner__core__plan_cost_ver2.t.a, 2), decimal(20,0) BINARY)->Column#9, planner__core__plan_cost_ver2.t.b->Column#10 - └─TableReader_12 100.00 35402.80 root data:TableFullScan_11 - └─TableFullScan_11 100.00 22730.80 cop[tikv] table:t keep order:false +HashAgg_7 100.00 295797.67 root group by:Column#10, funcs:sum(Column#8)->Column#4, funcs:sum(Column#9)->Column#5 +└─Projection_13 100.00 272740.67 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#8, cast(plus(planner__core__plan_cost_ver2.t.a, 2), decimal(20,0) BINARY)->Column#9, planner__core__plan_cost_ver2.t.b->Column#10 + └─TableReader_12 100.00 262710.77 root data:TableFullScan_11 + └─TableFullScan_11 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary), hash_agg() */ sum(a), sum(a+2), sum(a+4) from t group by b; id estRows estCost task access object operator info -HashAgg_7 100.00 74279.70 root group by:Column#13, funcs:sum(Column#10)->Column#4, funcs:sum(Column#11)->Column#5, funcs:sum(Column#12)->Column#6 -└─Projection_13 100.00 50422.70 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#10, cast(plus(planner__core__plan_cost_ver2.t.a, 2), decimal(20,0) BINARY)->Column#11, cast(plus(planner__core__plan_cost_ver2.t.a, 4), decimal(20,0) BINARY)->Column#12, planner__core__plan_cost_ver2.t.b->Column#13 - └─TableReader_12 100.00 35402.80 root data:TableFullScan_11 - └─TableFullScan_11 100.00 22730.80 cop[tikv] table:t keep order:false +HashAgg_7 100.00 301587.67 root group by:Column#13, funcs:sum(Column#10)->Column#4, funcs:sum(Column#11)->Column#5, funcs:sum(Column#12)->Column#6 +└─Projection_13 100.00 277730.67 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#10, cast(plus(planner__core__plan_cost_ver2.t.a, 2), decimal(20,0) BINARY)->Column#11, cast(plus(planner__core__plan_cost_ver2.t.a, 4), decimal(20,0) BINARY)->Column#12, planner__core__plan_cost_ver2.t.b->Column#13 + └─TableReader_12 100.00 262710.77 root data:TableFullScan_11 + └─TableFullScan_11 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary), hash_agg() */ sum(a) from t group by b; id estRows estCost task access object operator info -HashAgg_7 100.00 62699.70 root group by:Column#7, funcs:sum(Column#6)->Column#4 -└─Projection_13 100.00 40442.70 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#6, planner__core__plan_cost_ver2.t.b->Column#7 - └─TableReader_12 100.00 35402.80 root data:TableFullScan_11 - └─TableFullScan_11 100.00 22730.80 cop[tikv] table:t keep order:false +HashAgg_7 100.00 290007.67 root group by:Column#7, funcs:sum(Column#6)->Column#4 +└─Projection_13 100.00 267750.67 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#6, planner__core__plan_cost_ver2.t.b->Column#7 + └─TableReader_12 100.00 262710.77 root data:TableFullScan_11 + └─TableFullScan_11 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary), hash_agg() */ sum(a) from t group by b, b+1; id estRows estCost task access object operator info -HashAgg_7 100.00 77669.70 root group by:Column#8, Column#9, funcs:sum(Column#7)->Column#4 -└─Projection_13 100.00 45432.70 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#7, planner__core__plan_cost_ver2.t.b->Column#8, plus(planner__core__plan_cost_ver2.t.b, 1)->Column#9 - └─TableReader_12 100.00 35402.80 root data:TableFullScan_11 - └─TableFullScan_11 100.00 22730.80 cop[tikv] table:t keep order:false +HashAgg_7 100.00 304977.67 root group by:Column#8, Column#9, funcs:sum(Column#7)->Column#4 +└─Projection_13 100.00 272740.67 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#7, planner__core__plan_cost_ver2.t.b->Column#8, plus(planner__core__plan_cost_ver2.t.b, 1)->Column#9 + └─TableReader_12 100.00 262710.77 root data:TableFullScan_11 + └─TableFullScan_11 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary), hash_agg() */ sum(a) from t group by b, b+1, b+2; id estRows estCost task access object operator info -HashAgg_7 100.00 92639.70 root group by:Column#10, Column#11, Column#9, funcs:sum(Column#8)->Column#4 -└─Projection_13 100.00 50422.70 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#8, planner__core__plan_cost_ver2.t.b->Column#9, plus(planner__core__plan_cost_ver2.t.b, 1)->Column#10, plus(planner__core__plan_cost_ver2.t.b, 2)->Column#11 - └─TableReader_12 100.00 35402.80 root data:TableFullScan_11 - └─TableFullScan_11 100.00 22730.80 cop[tikv] table:t keep order:false +HashAgg_7 100.00 319947.67 root group by:Column#10, Column#11, Column#9, funcs:sum(Column#8)->Column#4 +└─Projection_13 100.00 277730.67 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#8, planner__core__plan_cost_ver2.t.b->Column#9, plus(planner__core__plan_cost_ver2.t.b, 1)->Column#10, plus(planner__core__plan_cost_ver2.t.b, 2)->Column#11 + └─TableReader_12 100.00 262710.77 root data:TableFullScan_11 + └─TableFullScan_11 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary), stream_agg() */ sum(a) from t group by b; id estRows estCost task access object operator info -StreamAgg_8 100.00 74235.54 root group by:Column#6, funcs:sum(Column#5)->Column#4 -└─Projection_17 100.00 74235.54 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#5, planner__core__plan_cost_ver2.t.b->Column#6 - └─Sort_13 100.00 69195.64 root planner__core__plan_cost_ver2.t.b - └─TableReader_12 100.00 35402.80 root data:TableFullScan_11 - └─TableFullScan_11 100.00 22730.80 cop[tikv] table:t keep order:false +StreamAgg_8 100.00 301543.51 root group by:Column#6, funcs:sum(Column#5)->Column#4 +└─Projection_17 100.00 301543.51 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#5, planner__core__plan_cost_ver2.t.b->Column#6 + └─Sort_13 100.00 296503.61 root planner__core__plan_cost_ver2.t.b + └─TableReader_12 100.00 262710.77 root data:TableFullScan_11 + └─TableFullScan_11 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary), stream_agg() */ sum(a), sum(a+2) from t group by b; id estRows estCost task access object operator info -StreamAgg_8 100.00 79225.54 root group by:Column#8, funcs:sum(Column#6)->Column#4, funcs:sum(Column#7)->Column#5 -└─Projection_17 100.00 79225.54 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#6, cast(plus(planner__core__plan_cost_ver2.t.a, 2), decimal(20,0) BINARY)->Column#7, planner__core__plan_cost_ver2.t.b->Column#8 - └─Sort_13 100.00 69195.64 root planner__core__plan_cost_ver2.t.b - └─TableReader_12 100.00 35402.80 root data:TableFullScan_11 - └─TableFullScan_11 100.00 22730.80 cop[tikv] table:t keep order:false +StreamAgg_8 100.00 306533.51 root group by:Column#8, funcs:sum(Column#6)->Column#4, funcs:sum(Column#7)->Column#5 +└─Projection_17 100.00 306533.51 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#6, cast(plus(planner__core__plan_cost_ver2.t.a, 2), decimal(20,0) BINARY)->Column#7, planner__core__plan_cost_ver2.t.b->Column#8 + └─Sort_13 100.00 296503.61 root planner__core__plan_cost_ver2.t.b + └─TableReader_12 100.00 262710.77 root data:TableFullScan_11 + └─TableFullScan_11 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ use_index(t, primary), stream_agg() */ sum(a), sum(a+2), sum(a+4) from t group by b; id estRows estCost task access object operator info -StreamAgg_8 100.00 84215.54 root group by:Column#10, funcs:sum(Column#7)->Column#4, funcs:sum(Column#8)->Column#5, funcs:sum(Column#9)->Column#6 -└─Projection_17 100.00 84215.54 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#7, cast(plus(planner__core__plan_cost_ver2.t.a, 2), decimal(20,0) BINARY)->Column#8, cast(plus(planner__core__plan_cost_ver2.t.a, 4), decimal(20,0) BINARY)->Column#9, planner__core__plan_cost_ver2.t.b->Column#10 - └─Sort_13 100.00 69195.64 root planner__core__plan_cost_ver2.t.b - └─TableReader_12 100.00 35402.80 root data:TableFullScan_11 - └─TableFullScan_11 100.00 22730.80 cop[tikv] table:t keep order:false +StreamAgg_8 100.00 311523.51 root group by:Column#10, funcs:sum(Column#7)->Column#4, funcs:sum(Column#8)->Column#5, funcs:sum(Column#9)->Column#6 +└─Projection_17 100.00 311523.51 root cast(planner__core__plan_cost_ver2.t.a, decimal(10,0) BINARY)->Column#7, cast(plus(planner__core__plan_cost_ver2.t.a, 2), decimal(20,0) BINARY)->Column#8, cast(plus(planner__core__plan_cost_ver2.t.a, 4), decimal(20,0) BINARY)->Column#9, planner__core__plan_cost_ver2.t.b->Column#10 + └─Sort_13 100.00 296503.61 root planner__core__plan_cost_ver2.t.b + └─TableReader_12 100.00 262710.77 root data:TableFullScan_11 + └─TableFullScan_11 100.00 250038.77 cop[tikv] table:t keep order:false explain format='verbose' select /*+ hash_join_build(t1) */ * from t t1, t t2 where t1.b=t2.b and t1.a<10; id estRows estCost task access object operator info HashJoin_35 10.00 63972.68 root inner join, equal:[eq(planner__core__plan_cost_ver2.t.b, planner__core__plan_cost_ver2.t.b)] @@ -254,7 +254,7 @@ explain format='true_card_cost' select * from t; Error 1105 (HY000): 'explain format=true_card_cost' cannot work without 'analyze', please use 'explain analyze format=true_card_cost' explain analyze format='true_card_cost' select * from t where a<3; id estRows estCost costFormula actRows task access object execution info operator info memory disk -TableReader_7 3323.33 135680.23 (((cpu(0*filters(1)*tikv_cpu_factor(49.9))) + ((scan(1*logrowsize(32)*tikv_scan_factor(40.7))) + (scan(10000*logrowsize(32)*tikv_scan_factor(40.7))))) + (net(0*rowsize(16)*tidb_kv_net_factor(3.96))))/15.00 0 root -└─Selection_6 3323.33 2035203.50 (cpu(0*filters(1)*tikv_cpu_factor(49.9))) + ((scan(1*logrowsize(32)*tikv_scan_factor(40.7))) + (scan(10000*logrowsize(32)*tikv_scan_factor(40.7)))) 0 cop[tikv] - └─TableFullScan_5 10000.00 2035203.50 (scan(1*logrowsize(32)*tikv_scan_factor(40.7))) + (scan(10000*logrowsize(32)*tikv_scan_factor(40.7))) 0 cop[tikv] table:t +TableReader_7 3323.33 13580.23 (((cpu(0*filters(1)*tikv_cpu_factor(49.9))) + ((scan(1*logrowsize(32)*tikv_scan_factor(40.7))) + (scan(1000*logrowsize(32)*tikv_scan_factor(40.7))))) + (net(0*rowsize(16)*tidb_kv_net_factor(3.96))))/15.00 0 root +└─Selection_6 3323.33 203703.50 (cpu(0*filters(1)*tikv_cpu_factor(49.9))) + ((scan(1*logrowsize(32)*tikv_scan_factor(40.7))) + (scan(1000*logrowsize(32)*tikv_scan_factor(40.7)))) 0 cop[tikv] + └─TableFullScan_5 10000.00 203703.50 (scan(1*logrowsize(32)*tikv_scan_factor(40.7))) + (scan(1000*logrowsize(32)*tikv_scan_factor(40.7))) 0 cop[tikv] table:t set @@tidb_cost_model_version=DEFAULT;