planner: Use/force to apply prefer range scan (#56928)
close pingcap/tidb#55632
This commit is contained in:
@ -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),
|
||||
}
|
||||
|
||||
@ -326,6 +326,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",
|
||||
|
||||
@ -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)"
|
||||
}
|
||||
]
|
||||
},
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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]"
|
||||
]
|
||||
},
|
||||
{
|
||||
|
||||
@ -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"
|
||||
]
|
||||
},
|
||||
{
|
||||
@ -445,8 +445,8 @@
|
||||
"Plan": [
|
||||
" TableReader root ",
|
||||
" └─ExchangeSender cop[tiflash] ",
|
||||
" └─Selection cop[tiflash] gt(test.t1.a, ?), or(lt(test.t1.a, ?), lt(test.t1.b, ?))",
|
||||
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.b, ?), keep order:false"
|
||||
" └─Selection cop[tiflash] gt(test.t1.b, ?), or(lt(test.t1.a, ?), lt(test.t1.b, ?))",
|
||||
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), keep order:false"
|
||||
]
|
||||
},
|
||||
{
|
||||
@ -454,8 +454,8 @@
|
||||
"Plan": [
|
||||
" TableReader root ",
|
||||
" └─ExchangeSender cop[tiflash] ",
|
||||
" └─Selection cop[tiflash] gt(test.t1.a, ?), gt(test.t1.c, ?), or(lt(test.t1.a, ?), lt(test.t1.b, ?))",
|
||||
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.b, ?), keep order:false"
|
||||
" └─Selection cop[tiflash] gt(test.t1.b, ?), gt(test.t1.c, ?), or(lt(test.t1.a, ?), lt(test.t1.b, ?))",
|
||||
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), 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"
|
||||
]
|
||||
},
|
||||
{
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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 <execution_info> <operator_info> <memory> <disk>
|
||||
└─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] <execution_info> <operator_info> <memory> <disk>
|
||||
└─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 <execution_info> <operator_info> <memory> <disk>
|
||||
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 <execution_info> <operator_info> <memory> <disk>
|
||||
└─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] <execution_info> <operator_info> <memory> <disk>
|
||||
└─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 <execution_info> <operator_info> <memory> <disk>
|
||||
set @@tidb_cost_model_version=DEFAULT;
|
||||
|
||||
Reference in New Issue
Block a user