Skip to content

Commit

Permalink
planner: support keep_order and no_keep_order hint (#40089)
Browse files Browse the repository at this point in the history
ref #39964
  • Loading branch information
Reminiscent committed Jan 9, 2023
1 parent 362defb commit b912237
Show file tree
Hide file tree
Showing 8 changed files with 168 additions and 33 deletions.
6 changes: 6 additions & 0 deletions parser/ast/dml.go
Original file line number Diff line number Diff line change
Expand Up @@ -358,6 +358,8 @@ const (
HintUse IndexHintType = iota + 1
HintIgnore
HintForce
HintKeepOrder
HintNoKeepOrder
)

// IndexHintScope is the type for index hint for join, order by or group by.
Expand Down Expand Up @@ -388,6 +390,10 @@ func (n *IndexHint) Restore(ctx *format.RestoreCtx) error {
indexHintType = "IGNORE INDEX"
case HintForce:
indexHintType = "FORCE INDEX"
case HintKeepOrder:
indexHintType = "KEEP ORDER"
case HintNoKeepOrder:
indexHintType = "NO KEEP ORDER"
default: // Prevent accidents
return errors.New("IndexHintType has an error while matching")
}
Expand Down
16 changes: 16 additions & 0 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -1436,6 +1436,14 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty,
if !prop.IsSortItemEmpty() && !candidate.isMatchProp {
return invalidTask, nil
}
// If we need to keep order for the index scan, we should forbid the non-keep-order index scan when we try to generate the path.
if prop.IsSortItemEmpty() && candidate.path.ForceKeepOrder {
return invalidTask, nil
}
// If we don't need to keep order for the index scan, we should forbid the non-keep-order index scan when we try to generate the path.
if !prop.IsSortItemEmpty() && candidate.path.ForceNoKeepOrder {
return invalidTask, nil
}
path := candidate.path
is := ds.getOriginalPhysicalIndexScan(prop, path, candidate.isMatchProp, candidate.path.IsSingleScan)
cop := &copTask{
Expand Down Expand Up @@ -1975,6 +1983,14 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid
if !prop.IsSortItemEmpty() && !candidate.isMatchProp {
return invalidTask, nil
}
// If we need to keep order for the index scan, we should forbid the non-keep-order index scan when we try to generate the path.
if prop.IsSortItemEmpty() && candidate.path.ForceKeepOrder {
return invalidTask, nil
}
// If we don't need to keep order for the index scan, we should forbid the non-keep-order index scan when we try to generate the path.
if !prop.IsSortItemEmpty() && candidate.path.ForceNoKeepOrder {
return invalidTask, nil
}
ts, _ := ds.getOriginalPhysicalTableScan(prop, candidate.path, candidate.isMatchProp)
if ts.KeepOrder && ts.StoreType == kv.TiFlash && (ts.Desc || ds.SCtx().GetSessionVars().TiFlashFastScan) {
// TiFlash fast mode(https://github.com/pingcap/tidb/pull/35851) does not keep order in TableScan
Expand Down
37 changes: 37 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1324,6 +1324,43 @@ func TestReadFromStorageHint(t *testing.T) {
}
}

func TestKeepOrderHint(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("set tidb_cost_model_version=2")
tk.MustExec("drop table if exists t, t1, t2")
tk.MustExec("create table t(a int, b int, primary key(a));")
tk.MustExec("create table t1(a int, b int, index idx_a(a), index idx_b(b));")

// If the optimizer can not generate the keep order plan, it will report error
err := tk.ExecToErr("explain select /*+ keep_order(t1, idx_a) */ * from t1 where a<10 limit 1;")
require.EqualError(t, err, "[planner:1815]Internal : Can't find a proper physical plan for this query")

err = tk.ExecToErr("explain select /*+ keep_order(t, primary) */ * from t where a<10 limit 1;")
require.EqualError(t, err, "[planner:1815]Internal : Can't find a proper physical plan for this query")

var input []string
var output []struct {
SQL string
Plan []string
Warn []string
}
integrationSuiteData := core.GetIntegrationSuiteData()
integrationSuiteData.LoadTestCases(t, &input, &output)
for i, tt := range input {
testdata.OnRecord(func() {
output[i].SQL = tt
output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows())
output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())
})
res := tk.MustQuery(tt)
res.Check(testkit.Rows(output[i].Plan...))
require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()))
}
}

func TestViewHint(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
Expand Down
51 changes: 19 additions & 32 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,10 @@ const (
HintIgnoreIndex = "ignore_index"
// HintForceIndex make optimizer to use this index even if it thinks a table scan is more efficient.
HintForceIndex = "force_index"
// HintKeepOrder is hint enforce using some indexes and keep the index's order.
HintKeepOrder = "keep_order"
// HintNoKeepOrder is hint enforce using some indexes and not keep the index's order.
HintNoKeepOrder = "no_keep_order"
// HintAggToCop is hint enforce pushing aggregation to coprocessor.
HintAggToCop = "agg_to_cop"
// HintReadFromStorage is hint enforce some tables read from specific type of storage.
Expand Down Expand Up @@ -3610,7 +3614,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev
// Set warning for the hint that requires the table name.
switch hint.HintName.L {
case TiDBMergeJoin, HintSMJ, TiDBIndexNestedLoopJoin, HintINLJ, HintINLHJ, HintINLMJ,
TiDBHashJoin, HintHJ, HintUseIndex, HintIgnoreIndex, HintForceIndex, HintIndexMerge, HintLeading:
TiDBHashJoin, HintHJ, HintUseIndex, HintIgnoreIndex, HintForceIndex, HintKeepOrder, HintNoKeepOrder, HintIndexMerge, HintLeading:
if len(hint.Tables) == 0 {
b.pushHintWithoutTableWarning(hint)
continue
Expand Down Expand Up @@ -3646,48 +3650,31 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev
aggHints.preferAggType |= preferStreamAgg
case HintAggToCop:
aggHints.preferAggToCop = true
case HintUseIndex:
case HintUseIndex, HintIgnoreIndex, HintForceIndex, HintKeepOrder, HintNoKeepOrder:
dbName := hint.Tables[0].DBName
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
}
indexHintList = append(indexHintList, indexHintInfo{
dbName: dbName,
tblName: hint.Tables[0].TableName,
partitions: hint.Tables[0].PartitionList,
indexHint: &ast.IndexHint{
IndexNames: hint.Indexes,
HintType: ast.HintUse,
HintScope: ast.HintForScan,
},
})
case HintIgnoreIndex:
dbName := hint.Tables[0].DBName
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
}
indexHintList = append(indexHintList, indexHintInfo{
dbName: dbName,
tblName: hint.Tables[0].TableName,
partitions: hint.Tables[0].PartitionList,
indexHint: &ast.IndexHint{
IndexNames: hint.Indexes,
HintType: ast.HintIgnore,
HintScope: ast.HintForScan,
},
})
case HintForceIndex:
dbName := hint.Tables[0].DBName
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
var hintType ast.IndexHintType
switch hint.HintName.L {
case HintUseIndex:
hintType = ast.HintUse
case HintIgnoreIndex:
hintType = ast.HintIgnore
case HintForceIndex:
hintType = ast.HintForce
case HintKeepOrder:
hintType = ast.HintKeepOrder
case HintNoKeepOrder:
hintType = ast.HintNoKeepOrder
}
indexHintList = append(indexHintList, indexHintInfo{
dbName: dbName,
tblName: hint.Tables[0].TableName,
partitions: hint.Tables[0].PartitionList,
indexHint: &ast.IndexHint{
IndexNames: hint.Indexes,
HintType: ast.HintForce,
HintType: hintType,
HintScope: ast.HintForScan,
},
})
Expand Down
6 changes: 6 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1421,6 +1421,12 @@ func getPossibleAccessPaths(ctx sessionctx.Context, tableHints *tableHintInfo, i
// our cost estimation is not reliable.
hasUseOrForce = true
path.Forced = true
if hint.HintType == ast.HintKeepOrder {
path.ForceKeepOrder = true
}
if hint.HintType == ast.HintNoKeepOrder {
path.ForceNoKeepOrder = true
}
available = append(available, path)
}
}
Expand Down
11 changes: 11 additions & 0 deletions planner/core/testdata/integration_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -653,6 +653,17 @@
"desc format = 'brief' select /*+ read_from_storage(tiflash[t, ttt], tikv[tt]) */ * from ttt"
]
},
{
"name": "TestKeepOrderHint",
"cases": [
"explain select /*+ keep_order(t1, idx_a) */ * from t1 where a<10 order by a limit 1;",
"explain select /*+ keep_order(t, primary) */ * from t where a<10 order by a limit 1;",
"explain select /*+ no_keep_order(t1, idx_a) */ * from t1 where a<10 order by a limit 1;",
"explain select /*+ no_keep_order(t, primary) */ * from t where a<10 order by a limit 1;",
"explain select /*+ no_keep_order(t1, idx_a) */ * from t1 where a<10 limit 1;",
"explain select /*+ no_keep_order(t, primary) */ * from t where a<10 limit 1;"
]
},
{
"name": "TestViewHint",
"cases": [
Expand Down
68 changes: 68 additions & 0 deletions planner/core/testdata/integration_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -4321,6 +4321,74 @@
}
]
},
{
"Name": "TestKeepOrderHint",
"Cases": [
{
"SQL": "explain select /*+ keep_order(t1, idx_a) */ * from t1 where a<10 order by a limit 1;",
"Plan": [
"Limit_12 1.00 root offset:0, count:1",
"└─Projection_17 1.00 root test.t1.a, test.t1.b",
" └─IndexLookUp_16 1.00 root ",
" ├─Limit_15(Build) 1.00 cop[tikv] offset:0, count:1",
" │ └─IndexRangeScan_13 1.00 cop[tikv] table:t1, index:idx_a(a) range:[-inf,10), keep order:true, stats:pseudo",
" └─TableRowIDScan_14(Probe) 1.00 cop[tikv] table:t1 keep order:false, stats:pseudo"
],
"Warn": null
},
{
"SQL": "explain select /*+ keep_order(t, primary) */ * from t where a<10 order by a limit 1;",
"Plan": [
"Limit_11 1.00 root offset:0, count:1",
"└─TableReader_15 1.00 root data:Limit_14",
" └─Limit_14 1.00 cop[tikv] offset:0, count:1",
" └─TableRangeScan_13 333.33 cop[tikv] table:t range:[-inf,10), keep order:true, stats:pseudo"
],
"Warn": null
},
{
"SQL": "explain select /*+ no_keep_order(t1, idx_a) */ * from t1 where a<10 order by a limit 1;",
"Plan": [
"TopN_9 1.00 root test.t1.a, offset:0, count:1",
"└─IndexLookUp_16 1.00 root ",
" ├─TopN_15(Build) 1.00 cop[tikv] test.t1.a, offset:0, count:1",
" │ └─IndexRangeScan_13 3323.33 cop[tikv] table:t1, index:idx_a(a) range:[-inf,10), keep order:false, stats:pseudo",
" └─TableRowIDScan_14(Probe) 1.00 cop[tikv] table:t1 keep order:false, stats:pseudo"
],
"Warn": null
},
{
"SQL": "explain select /*+ no_keep_order(t, primary) */ * from t where a<10 order by a limit 1;",
"Plan": [
"TopN_8 1.00 root test.t.a, offset:0, count:1",
"└─TableReader_15 1.00 root data:TopN_14",
" └─TopN_14 1.00 cop[tikv] test.t.a, offset:0, count:1",
" └─TableRangeScan_13 3333.33 cop[tikv] table:t range:[-inf,10), keep order:false, stats:pseudo"
],
"Warn": null
},
{
"SQL": "explain select /*+ no_keep_order(t1, idx_a) */ * from t1 where a<10 limit 1;",
"Plan": [
"IndexLookUp_13 1.00 root limit embedded(offset:0, count:1)",
"├─Limit_12(Build) 1.00 cop[tikv] offset:0, count:1",
"│ └─IndexRangeScan_10 1.00 cop[tikv] table:t1, index:idx_a(a) range:[-inf,10), keep order:false, stats:pseudo",
"└─TableRowIDScan_11(Probe) 1.00 cop[tikv] table:t1 keep order:false, stats:pseudo"
],
"Warn": null
},
{
"SQL": "explain select /*+ no_keep_order(t, primary) */ * from t where a<10 limit 1;",
"Plan": [
"Limit_8 1.00 root offset:0, count:1",
"└─TableReader_12 1.00 root data:Limit_11",
" └─Limit_11 1.00 cop[tikv] offset:0, count:1",
" └─TableRangeScan_10 333.33 cop[tikv] table:t range:[-inf,10), keep order:false, stats:pseudo"
],
"Warn": null
}
]
},
{
"Name": "TestViewHint",
"Cases": [
Expand Down
6 changes: 5 additions & 1 deletion planner/util/path.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,9 @@ type AccessPath struct {
IsIntHandlePath bool
IsCommonHandlePath bool
// Forced means this path is generated by `use/force index()`.
Forced bool
Forced bool
ForceKeepOrder bool
ForceNoKeepOrder bool
// IsSingleScan indicates whether the path is a single index/table scan or table access after index scan.
IsSingleScan bool

Expand Down Expand Up @@ -97,6 +99,8 @@ func (path *AccessPath) Clone() *AccessPath {
IsIntHandlePath: path.IsIntHandlePath,
IsCommonHandlePath: path.IsCommonHandlePath,
Forced: path.Forced,
ForceKeepOrder: path.ForceKeepOrder,
ForceNoKeepOrder: path.ForceNoKeepOrder,
IsSingleScan: path.IsSingleScan,
IsUkShardIndexPath: path.IsUkShardIndexPath,
}
Expand Down

0 comments on commit b912237

Please sign in to comment.