Skip to content

Commit cdfdb8a

Browse files
planner: Use/force to apply prefer range scan (#56928)
close #55632
1 parent 243e9c6 commit cdfdb8a

File tree

13 files changed

+228
-127
lines changed

13 files changed

+228
-127
lines changed

pkg/executor/test/issuetest/executor_issue_test.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -321,12 +321,14 @@ func TestIndexJoin31494(t *testing.T) {
321321
insertStr += fmt.Sprintf(", (%d, %d)", i, i)
322322
}
323323
tk.MustExec(insertStr)
324+
tk.MustExec("analyze table t1")
324325
tk.MustExec("create table t2(a int(11) default null, b int(11) default null, c int(11) default null)")
325326
insertStr = "insert into t2 values(1, 1, 1)"
326327
for i := 1; i < 32768; i++ {
327328
insertStr += fmt.Sprintf(", (%d, %d, %d)", i, i, i)
328329
}
329330
tk.MustExec(insertStr)
331+
tk.MustExec("analyze table t2")
330332
sm := &testkit.MockSessionManager{
331333
PS: make([]*util.ProcessInfo, 0),
332334
}

pkg/planner/core/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -326,6 +326,7 @@ go_test(
326326
"//pkg/util/ranger",
327327
"//pkg/util/stmtsummary",
328328
"//pkg/util/tracing",
329+
"//tests/realtikvtest",
329330
"@com_github_docker_go_units//:go-units",
330331
"@com_github_golang_snappy//:snappy",
331332
"@com_github_pingcap_errors//:errors",

pkg/planner/core/casetest/dag/testdata/plan_suite_out.json

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,7 @@
4848
},
4949
{
5050
"SQL": "select c from t order by t.a limit 1",
51-
"Best": "IndexReader(Index(t.c_d_e)[[NULL,+inf]]->TopN([test.t.a],0,1))->TopN([test.t.a],0,1)->Projection"
51+
"Best": "TableReader(Table(t)->Limit)->Limit->Projection"
5252
},
5353
{
5454
"SQL": "select c from t order by t.a + t.b limit 1",
@@ -165,7 +165,7 @@
165165
},
166166
{
167167
"SQL": "select * from t t1 join t t2 on t1.b = t2.a order by t1.a",
168-
"Best": "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.b,test.t.a)->Sort"
168+
"Best": "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.b,test.t.a)"
169169
},
170170
{
171171
"SQL": "select * from t t1 join t t2 on t1.b = t2.a order by t1.a limit 1",
@@ -508,7 +508,7 @@
508508
},
509509
{
510510
"SQL": "select a from t union all (select c from t) order by a limit 1",
511-
"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)"
511+
"Best": "UnionAll{TableReader(Table(t)->Limit)->Limit->IndexReader(Index(t.c_d_e)[[NULL,+inf]]->Limit)->Limit}->TopN([Column#25],0,1)"
512512
}
513513
]
514514
},

pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -2257,11 +2257,11 @@
22572257
"Cases": [
22582258
{
22592259
"SQL": "select max(a) from t;",
2260-
"Best": "IndexReader(Index(t.f)[[NULL,+inf]]->TopN([test.t.a true],0,1))->TopN([test.t.a true],0,1)->StreamAgg"
2260+
"Best": "TableReader(Table(t)->Limit)->Limit->StreamAgg"
22612261
},
22622262
{
22632263
"SQL": "select min(a) from t;",
2264-
"Best": "IndexReader(Index(t.f)[[NULL,+inf]]->TopN([test.t.a],0,1))->TopN([test.t.a],0,1)->StreamAgg"
2264+
"Best": "TableReader(Table(t)->Limit)->Limit->StreamAgg"
22652265
},
22662266
{
22672267
"SQL": "select min(c_str) from t;",
@@ -2277,7 +2277,7 @@
22772277
},
22782278
{
22792279
"SQL": "select max(a), min(a) from t;",
2280-
"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}"
2280+
"Best": "LeftHashJoin{TableReader(Table(t)->Limit)->Limit->StreamAgg->TableReader(Table(t)->Limit)->Limit->StreamAgg}"
22812281
},
22822282
{
22832283
"SQL": "select max(a), min(a) from t where a > 10",
@@ -2289,7 +2289,7 @@
22892289
},
22902290
{
22912291
"SQL": "select max(a), max(c), min(f) from t",
2292-
"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}"
2292+
"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}"
22932293
},
22942294
{
22952295
"SQL": "select max(a), max(b) from t",

pkg/planner/core/casetest/planstats/testdata/plan_stats_suite_out.json

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -125,13 +125,14 @@
125125
"Query": "explain format = brief select * from t join tp where tp.a = 10 and t.b = tp.c",
126126
"Result": [
127127
"Projection 1.00 root test.t.a, test.t.b, test.t.c, test.tp.a, test.tp.b, test.tp.c",
128-
"└─HashJoin 1.00 root inner join, equal:[eq(test.tp.c, test.t.b)]",
128+
"└─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)",
129129
" ├─TableReader(Build) 1.00 root partition:p1 data:Selection",
130130
" │ └─Selection 1.00 cop[tikv] eq(test.tp.a, 10), not(isnull(test.tp.c))",
131131
" │ └─TableFullScan 6.00 cop[tikv] table:tp keep order:false, stats:partial[c:allEvicted]",
132-
" └─TableReader(Probe) 3.00 root data:Selection",
133-
" └─Selection 3.00 cop[tikv] not(isnull(test.t.b))",
134-
" └─TableFullScan 3.00 cop[tikv] table:t keep order:false, stats:partial[idx:allEvicted, a:allEvicted, b:allEvicted]"
132+
" └─IndexLookUp(Probe) 1.00 root ",
133+
" ├─Selection(Build) 1.00 cop[tikv] not(isnull(test.t.b))",
134+
" │ └─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]",
135+
" └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false, stats:partial[idx:allEvicted, a:allEvicted, b:allEvicted]"
135136
]
136137
},
137138
{

pkg/planner/core/casetest/testdata/plan_normalized_suite_out.json

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -428,8 +428,8 @@
428428
"Plan": [
429429
" TableReader root ",
430430
" └─ExchangeSender cop[tiflash] ",
431-
" └─Selection cop[tiflash] gt(test.t1.a, ?)",
432-
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.b, ?), gt(test.t1.c, ?), keep order:false"
431+
" └─Selection cop[tiflash] gt(test.t1.c, ?)",
432+
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), gt(test.t1.b, ?), keep order:false"
433433
]
434434
},
435435
{
@@ -445,26 +445,26 @@
445445
"Plan": [
446446
" TableReader root ",
447447
" └─ExchangeSender cop[tiflash] ",
448-
" └─Selection cop[tiflash] gt(test.t1.a, ?), or(lt(test.t1.a, ?), lt(test.t1.b, ?))",
449-
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.b, ?), keep order:false"
448+
" └─Selection cop[tiflash] gt(test.t1.b, ?), or(lt(test.t1.a, ?), lt(test.t1.b, ?))",
449+
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), keep order:false"
450450
]
451451
},
452452
{
453453
"SQL": "explain select * from t1 where (a<1 or b<2) and (a>3 and b>3) and c>1",
454454
"Plan": [
455455
" TableReader root ",
456456
" └─ExchangeSender cop[tiflash] ",
457-
" └─Selection cop[tiflash] gt(test.t1.a, ?), gt(test.t1.c, ?), or(lt(test.t1.a, ?), lt(test.t1.b, ?))",
458-
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.b, ?), keep order:false"
457+
" └─Selection cop[tiflash] gt(test.t1.b, ?), gt(test.t1.c, ?), or(lt(test.t1.a, ?), lt(test.t1.b, ?))",
458+
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), keep order:false"
459459
]
460460
},
461461
{
462462
"SQL": "explain select * from t1 where (a>2 or b<2) and (a>3 and b>3) and c>2",
463463
"Plan": [
464464
" TableReader root ",
465465
" └─ExchangeSender cop[tiflash] ",
466-
" └─Selection cop[tiflash] gt(test.t1.b, ?), gt(test.t1.c, ?), or(gt(test.t1.a, ?), lt(test.t1.b, ?))",
467-
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.a, ?), keep order:false"
466+
" └─Selection cop[tiflash] gt(test.t1.a, ?), gt(test.t1.c, ?), or(gt(test.t1.a, ?), lt(test.t1.b, ?))",
467+
" └─TableFullScan cop[tiflash] table:t1, range:[?,?], pushed down filter:gt(test.t1.b, ?), keep order:false"
468468
]
469469
},
470470
{

pkg/planner/core/find_best_task.go

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1172,13 +1172,15 @@ func skylinePruning(ds *logicalop.DataSource, prop *property.PhysicalProperty) [
11721172
}
11731173
}
11741174

1175-
preferRange := ds.SCtx().GetSessionVars().GetAllowPreferRangeScan() && (ds.TableStats.HistColl.Pseudo || ds.TableStats.RowCount < 1)
11761175
// If we've forced an index merge - we want to keep these plans
11771176
preferMerge := len(ds.IndexMergeHints) > 0 || fixcontrol.GetBoolWithDefault(
11781177
ds.SCtx().GetSessionVars().GetOptimizerFixControlMap(),
11791178
fixcontrol.Fix52869,
11801179
false,
11811180
)
1181+
// tidb_opt_prefer_range_scan is the master switch to control index preferencing
1182+
preferRange := ds.SCtx().GetSessionVars().GetAllowPreferRangeScan() &&
1183+
(preferMerge || (ds.TableStats.HistColl.Pseudo || ds.TableStats.RowCount < 1))
11821184
if preferRange && len(candidates) > 1 {
11831185
// If a candidate path is TiFlash-path or forced-path or MV index, we just keep them. For other candidate paths, if there exists
11841186
// 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) [
11971199
}
11981200
if !ranger.HasFullRange(c.path.Ranges, unsignedIntHandle) {
11991201
// Preference plans with equals/IN predicates or where there is more filtering in the index than against the table
1200-
equalPlan := c.path.EqCondCount > 0 || c.path.EqOrInCondCount > 0
1201-
indexFilters := len(c.path.TableFilters) < len(c.path.IndexFilters)
1202-
if preferMerge || (((equalPlan || indexFilters) && prop.IsSortItemEmpty()) || c.isMatchProp) {
1202+
indexFilters := c.path.EqCondCount > 0 || c.path.EqOrInCondCount > 0 || len(c.path.TableFilters) < len(c.path.IndexFilters)
1203+
if preferMerge || (indexFilters && (prop.IsSortItemEmpty() || c.isMatchProp)) {
12031204
preferredPaths = append(preferredPaths, c)
12041205
hasRangeScanPath = true
12051206
}

pkg/planner/core/logical_plans_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2020,7 +2020,7 @@ func TestSkylinePruning(t *testing.T) {
20202020
},
20212021
{
20222022
sql: "select * from pt2_global_index where b > 1 order by b",
2023-
result: "b_global,b_c_global",
2023+
result: "PRIMARY_KEY,b_global,b_c_global",
20242024
},
20252025
{
20262026
sql: "select b from pt2_global_index where b > 1 order by b",

pkg/planner/core/plan_cost_ver2.go

Lines changed: 61 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -163,30 +163,8 @@ func (p *PhysicalTableScan) GetPlanCostVer2(taskType property.TaskType, option *
163163
if p.StoreType == kv.TiFlash {
164164
p.PlanCostVer2 = costusage.SumCostVer2(p.PlanCostVer2, scanCostVer2(option, TiFlashStartupRowPenalty, rowSize, scanFactor))
165165
} else if !p.isChildOfIndexLookUp {
166-
// Apply cost penalty for full scans that carry high risk of underestimation
167-
sessionVars := p.SCtx().GetSessionVars()
168-
allowPreferRangeScan := sessionVars.GetAllowPreferRangeScan()
169-
tblColHists := p.tblColHists
170-
171-
// hasUnreliableStats is a check for pseudo or zero stats
172-
hasUnreliableStats := tblColHists.Pseudo || tblColHists.RealtimeCount < 1
173-
// hasHighModifyCount tracks the high risk of a tablescan where auto-analyze had not yet updated the table row count
174-
hasHighModifyCount := tblColHists.ModifyCount > tblColHists.RealtimeCount
175-
// hasLowEstimate is a check to capture a unique customer case where modifyCount is used for tablescan estimate (but it not adequately understood why)
176-
hasLowEstimate := rows > 1 && tblColHists.ModifyCount < tblColHists.RealtimeCount && int64(rows) <= tblColHists.ModifyCount
177-
// preferRangeScan check here is same as in skylinePruning
178-
preferRangeScanCondition := allowPreferRangeScan && (hasUnreliableStats || hasHighModifyCount || hasLowEstimate)
179-
var unsignedIntHandle bool
180-
if p.Table.PKIsHandle {
181-
if pkColInfo := p.Table.GetPkColInfo(); pkColInfo != nil {
182-
unsignedIntHandle = mysql.HasUnsignedFlag(pkColInfo.GetFlag())
183-
}
184-
}
185-
hasFullRangeScan := ranger.HasFullRange(p.Ranges, unsignedIntHandle)
186-
187-
shouldApplyPenalty := hasFullRangeScan && preferRangeScanCondition
188-
if shouldApplyPenalty {
189-
newRowCount := max(MaxPenaltyRowCount, max(float64(tblColHists.ModifyCount), float64(tblColHists.RealtimeCount)))
166+
newRowCount := getTableScanPenalty(p, rows)
167+
if newRowCount > 0 {
190168
p.PlanCostVer2 = costusage.SumCostVer2(p.PlanCostVer2, scanCostVer2(option, newRowCount, rowSize, scanFactor))
191169
}
192170
}
@@ -935,6 +913,65 @@ func doubleReadCostVer2(option *optimizetrace.PlanCostOption, numTasks float64,
935913
func() string { return fmt.Sprintf("doubleRead(tasks(%v)*%v)", numTasks, requestFactor) })
936914
}
937915

916+
func getTableScanPenalty(p *PhysicalTableScan, rows float64) (rowPenalty float64) {
917+
// Apply cost penalty for full scans that carry high risk of underestimation. Exclude those
918+
// that are the child of an index scan or child is TableRangeScan
919+
if len(p.rangeInfo) > 0 {
920+
return float64(0)
921+
}
922+
var unsignedIntHandle bool
923+
if p.Table.PKIsHandle {
924+
if pkColInfo := p.Table.GetPkColInfo(); pkColInfo != nil {
925+
unsignedIntHandle = mysql.HasUnsignedFlag(pkColInfo.GetFlag())
926+
}
927+
}
928+
hasFullRangeScan := ranger.HasFullRange(p.Ranges, unsignedIntHandle)
929+
if !hasFullRangeScan {
930+
return float64(0)
931+
}
932+
933+
sessionVars := p.SCtx().GetSessionVars()
934+
allowPreferRangeScan := sessionVars.GetAllowPreferRangeScan()
935+
tblColHists := p.tblColHists
936+
originalRows := int64(tblColHists.GetAnalyzeRowCount())
937+
938+
// hasUnreliableStats is a check for pseudo or zero stats
939+
hasUnreliableStats := tblColHists.Pseudo || originalRows < 1
940+
// hasHighModifyCount tracks the high risk of a tablescan where auto-analyze had not yet updated the table row count
941+
hasHighModifyCount := tblColHists.ModifyCount > originalRows
942+
// hasLowEstimate is a check to capture a unique customer case where modifyCount is used for tablescan estimate (but it not adequately understood why)
943+
hasLowEstimate := rows > 1 && tblColHists.ModifyCount < originalRows && int64(rows) <= tblColHists.ModifyCount
944+
// preferRangeScan check here is same as in skylinePruning
945+
preferRangeScanCondition := allowPreferRangeScan && (hasUnreliableStats || hasHighModifyCount || hasLowEstimate)
946+
947+
// differentiate a FullTableScan from a partition level scan - so we shouldn't penalize these
948+
hasPartitionScan := false
949+
if p.PlanPartInfo != nil {
950+
if len(p.PlanPartInfo.PruningConds) > 0 {
951+
hasPartitionScan = true
952+
}
953+
}
954+
955+
// GetIndexForce assumes that the USE/FORCE index is to force a range scan, and thus the
956+
// penalty is applied to a full table scan (not range scan). This may also penalize a
957+
// full table scan where USE/FORCE was applied to the primary key.
958+
hasIndexForce := sessionVars.StmtCtx.GetIndexForce()
959+
shouldApplyPenalty := hasFullRangeScan && (hasIndexForce || preferRangeScanCondition)
960+
if shouldApplyPenalty {
961+
// MySQL will increase the cost of table scan if FORCE index is used. TiDB takes this one
962+
// step further - because we don't differentiate USE/FORCE - the added penalty applies to
963+
// both, and it also applies to any full table scan in the query. Use "max" to get the minimum
964+
// number of rows to add as a penalty to the table scan.
965+
minRows := max(MaxPenaltyRowCount, rows)
966+
if hasPartitionScan {
967+
return minRows
968+
}
969+
// If it isn't a partitioned table - choose the max that includes ModifyCount
970+
return max(minRows, float64(tblColHists.ModifyCount))
971+
}
972+
return float64(0)
973+
}
974+
938975
// In Cost Ver2, we hide cost factors from users and deprecate SQL variables like `tidb_opt_scan_factor`.
939976
type costVer2Factors struct {
940977
TiDBTemp costusage.CostVer2Factor // operations on TiDB temporary table

pkg/planner/core/plan_cost_ver2_test.go

Lines changed: 43 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ import (
3333
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
3434
"github.com/pingcap/tidb/pkg/sessiontxn"
3535
"github.com/pingcap/tidb/pkg/testkit"
36+
"github.com/pingcap/tidb/tests/realtikvtest"
3637
"github.com/stretchr/testify/require"
3738
)
3839

@@ -59,9 +60,9 @@ func TestCostModelVer2ScanRowSize(t *testing.T) {
5960
{"select a, b from t use index(abc) where a=1 and b=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"},
6061
{"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))"},
6162
// table scan row-size is always equal to row-size(*)
62-
{"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)))"},
63-
{"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)))"},
64-
{"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)))"},
63+
{"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)))"},
64+
{"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)))"},
65+
{"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)))"},
6566
}
6667
for _, c := range cases {
6768
rs := tk.MustQuery("explain analyze format=true_card_cost " + c.query).Rows()
@@ -162,3 +163,42 @@ func BenchmarkGetPlanCost(b *testing.B) {
162163
_, _ = core.GetPlanCost(phyPlan, property.RootTaskType, optimizetrace.NewDefaultPlanCostOption().WithCostFlag(costusage.CostFlagRecalculate))
163164
}
164165
}
166+
167+
func TestTableScanCostWithForce(t *testing.T) {
168+
store, dom := realtikvtest.CreateMockStoreAndDomainAndSetup(t)
169+
defer func() {
170+
tk := testkit.NewTestKit(t, store)
171+
tk.MustExec("use test")
172+
tk.MustExec("drop table if exists t")
173+
dom.StatsHandle().Clear()
174+
}()
175+
176+
tk := testkit.NewTestKit(t, store)
177+
tk.MustExec("use test")
178+
tk.MustExec("drop table if exists t")
179+
tk.MustExec("create table t(a int, b int, primary key (a))")
180+
181+
// Insert some data
182+
tk.MustExec("insert into t values (1,1),(2,2),(3,3),(4,4),(5,5)")
183+
184+
// Analyze table to update statistics
185+
tk.MustExec("analyze table t")
186+
187+
// Test TableFullScan with and without FORCE INDEX
188+
rs := tk.MustQuery("explain analyze format=verbose select * from t").Rows()
189+
planCost1 := rs[0][2].(string)
190+
rs = tk.MustQuery("explain analyze format=verbose select * from t force index(PRIMARY)").Rows()
191+
planCost2 := rs[0][2].(string)
192+
193+
// Query with FORCE should be more expensive than query without
194+
require.Less(t, planCost1, planCost2)
195+
196+
// Test TableRangeScan with and without FORCE INDEX
197+
rs = tk.MustQuery("explain analyze format=verbose select * from t where a > 1").Rows()
198+
planCost1 = rs[0][2].(string)
199+
rs = tk.MustQuery("explain analyze format=verbose select * from t force index(PRIMARY) where a > 1").Rows()
200+
planCost2 = rs[0][2].(string)
201+
202+
// Query costs should be equal since FORCE cost penalty does not apply to range scan
203+
require.Equal(t, planCost1, planCost2)
204+
}

0 commit comments

Comments
 (0)