Skip to content

Commit

Permalink
planner: add warning when the table name of indexHint cannot be found (
Browse files Browse the repository at this point in the history
  • Loading branch information
francis0407 authored and sre-bot committed Mar 27, 2020
1 parent ff0c182 commit 2cf7a2c
Show file tree
Hide file tree
Showing 7 changed files with 174 additions and 7 deletions.
33 changes: 33 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testutil"
)
Expand Down Expand Up @@ -625,6 +626,38 @@ func (s *testIntegrationSuite) TestSubqueryWithTopN(c *C) {
}
}

func (s *testIntegrationSuite) TestIndexHintWarning(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1(a int, b int, c int, key a(a))")
tk.MustExec("create table t2(a int, b int, c int, key a(a))")
var input []string
var output []struct {
SQL string
Warnings []string
}
s.testData.GetTestCases(c, &input, &output)
for i, tt := range input {
s.testData.OnRecord(func() {
output[i].SQL = tt
tk.MustQuery(tt)
warns := tk.Se.GetSessionVars().StmtCtx.GetWarnings()
output[i].Warnings = make([]string, len(warns))
for j := range warns {
output[i].Warnings[j] = warns[j].Err.Error()
}
})
tk.MustQuery(tt)
warns := tk.Se.GetSessionVars().StmtCtx.GetWarnings()
c.Assert(len(warns), Equals, len(output[i].Warnings))
for j := range warns {
c.Assert(warns[j].Level, Equals, stmtctx.WarnLevelWarning)
c.Assert(warns[j].Err.Error(), Equals, output[i].Warnings[j])
}
}
}

func (s *testIntegrationSuite) TestIssue15546(c *C) {
tk := testkit.NewTestKit(c, s.store)

Expand Down
30 changes: 29 additions & 1 deletion planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2251,7 +2251,12 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, nodeType n
}
case HintIndexMerge:
if len(hint.Tables) != 0 {
dbName := hint.Tables[0].DBName
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
}
indexMergeHintList = append(indexMergeHintList, indexHintInfo{
dbName: dbName,
tblName: hint.Tables[0].TableName,
indexHint: &ast.IndexHint{
IndexNames: hint.Indexes,
Expand Down Expand Up @@ -2281,6 +2286,8 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, nodeType n

func (b *PlanBuilder) popTableHints() {
hintInfo := b.tableHintInfo[len(b.tableHintInfo)-1]
b.appendUnmatchedIndexHintWarning(hintInfo.indexHintList, false)
b.appendUnmatchedIndexHintWarning(hintInfo.indexMergeHintList, true)
b.appendUnmatchedJoinHintWarning(HintINLJ, TiDBIndexNestedLoopJoin, hintInfo.indexNestedLoopJoinTables.inljTables)
b.appendUnmatchedJoinHintWarning(HintINLHJ, "", hintInfo.indexNestedLoopJoinTables.inlhjTables)
b.appendUnmatchedJoinHintWarning(HintINLMJ, "", hintInfo.indexNestedLoopJoinTables.inlmjTables)
Expand All @@ -2289,6 +2296,26 @@ func (b *PlanBuilder) popTableHints() {
b.tableHintInfo = b.tableHintInfo[:len(b.tableHintInfo)-1]
}

func (b *PlanBuilder) appendUnmatchedIndexHintWarning(indexHints []indexHintInfo, usedForIndexMerge bool) {
for _, hint := range indexHints {
if !hint.matched {
var hintTypeString string
if usedForIndexMerge {
hintTypeString = "use_index_merge"
} else {
hintTypeString = hint.hintTypeString()
}
errMsg := fmt.Sprintf("%s(%s) is inapplicable, check whether the table(%s.%s) exists",
hintTypeString,
hint.indexString(),
hint.dbName,
hint.tblName,
)
b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(errMsg))
}
}
}

func (b *PlanBuilder) appendUnmatchedJoinHintWarning(joinType string, joinTypeAlias string, hintTables []hintTableInfo) {
unMatchedTables := extractUnmatchedTables(hintTables)
if len(unMatchedTables) == 0 {
Expand Down Expand Up @@ -2629,9 +2656,10 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as
// extract the IndexMergeHint
var indexMergeHints []*ast.IndexHint
if hints := b.TableHints(); hints != nil {
for _, hint := range hints.indexMergeHintList {
for i, hint := range hints.indexMergeHintList {
if hint.tblName.L == tblName.L {
indexMergeHints = append(indexMergeHints, hint.indexHint)
hints.indexMergeHintList[i].matched = true
}
}
}
Expand Down
33 changes: 32 additions & 1 deletion planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,36 @@ type indexHintInfo struct {
dbName model.CIStr
tblName model.CIStr
indexHint *ast.IndexHint
// Matched indicates whether this index hint
// has been successfully applied to a DataSource.
// If an indexHintInfo is not matched after building
// a Select statement, we will generate a warning for it.
matched bool
}

func (hint *indexHintInfo) hintTypeString() string {
switch hint.indexHint.HintType {
case ast.HintUse:
return "use_index"
case ast.HintIgnore:
return "ignore_index"
case ast.HintForce:
return "force_index"
}
return ""
}

// indexString formats the indexHint as dbName.tableName[, indexNames].
func (hint *indexHintInfo) indexString() string {
var indexListString string
indexList := make([]string, len(hint.indexHint.IndexNames))
for i := range hint.indexHint.IndexNames {
indexList[i] = hint.indexHint.IndexNames[i].L
}
if len(indexList) > 0 {
indexListString = fmt.Sprintf(", %s", strings.Join(indexList, ", "))
}
return fmt.Sprintf("%s.%s%s", hint.dbName, hint.tblName, indexListString)
}

type aggHintInfo struct {
Expand Down Expand Up @@ -670,9 +700,10 @@ func (b *PlanBuilder) getPossibleAccessPaths(indexHints []*ast.IndexHint, tbl ta
// Extract comment-style index hint like /*+ INDEX(t, idx1, idx2) */.
indexHintsLen := len(indexHints)
if hints := b.TableHints(); hints != nil {
for _, hint := range hints.indexHintList {
for i, hint := range hints.indexHintList {
if hint.dbName.L == dbName.L && hint.tblName.L == tblName.L {
indexHints = append(indexHints, hint.indexHint)
hints.indexHintList[i].matched = true
}
}
}
Expand Down
12 changes: 12 additions & 0 deletions planner/core/testdata/integration_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -93,5 +93,17 @@
"desc select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.b",
"desc select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.a and t1.b = t2.b"
]
},
{
"name": "TestIndexHintWarning",
"cases": [
"select /*+ USE_INDEX(t1, j) */ * from t1",
"select /*+ IGNORE_INDEX(t1, j) */ * from t1",
"select /*+ USE_INDEX(t2, a, b, c) */ * from t1",
"select /*+ USE_INDEX(t2) */ * from t1",
"select /*+ USE_INDEX(t1, a), USE_INDEX(t2, a), USE_INDEX(t3, a) */ * from t1, t2 where t1.a=t2.a",
"select /*+ USE_INDEX(t3, a), USE_INDEX(t4, b), IGNORE_INDEX(t3, a) */ * from t1, t2 where t1.a=t2.a",
"select /*+ USE_INDEX_MERGE(t3, a, b, d) */ * from t1"
]
}
]
49 changes: 49 additions & 0 deletions planner/core/testdata/integration_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -389,5 +389,54 @@
]
}
]
},
{
"Name": "TestIndexHintWarning",
"Cases": [
{
"SQL": "select /*+ USE_INDEX(t1, j) */ * from t1",
"Warnings": [
"[planner:1176]Key 'j' doesn't exist in table 't1'"
]
},
{
"SQL": "select /*+ IGNORE_INDEX(t1, j) */ * from t1",
"Warnings": [
"[planner:1176]Key 'j' doesn't exist in table 't1'"
]
},
{
"SQL": "select /*+ USE_INDEX(t2, a, b, c) */ * from t1",
"Warnings": [
"[planner:1815]use_index(test.t2, a, b, c) is inapplicable, check whether the table(test.t2) exists"
]
},
{
"SQL": "select /*+ USE_INDEX(t2) */ * from t1",
"Warnings": [
"[planner:1815]use_index(test.t2) is inapplicable, check whether the table(test.t2) exists"
]
},
{
"SQL": "select /*+ USE_INDEX(t1, a), USE_INDEX(t2, a), USE_INDEX(t3, a) */ * from t1, t2 where t1.a=t2.a",
"Warnings": [
"[planner:1815]use_index(test.t3, a) is inapplicable, check whether the table(test.t3) exists"
]
},
{
"SQL": "select /*+ USE_INDEX(t3, a), USE_INDEX(t4, b), IGNORE_INDEX(t3, a) */ * from t1, t2 where t1.a=t2.a",
"Warnings": [
"[planner:1815]use_index(test.t3, a) is inapplicable, check whether the table(test.t3) exists",
"[planner:1815]use_index(test.t4, b) is inapplicable, check whether the table(test.t4) exists",
"[planner:1815]ignore_index(test.t3, a) is inapplicable, check whether the table(test.t3) exists"
]
},
{
"SQL": "select /*+ USE_INDEX_MERGE(t3, a, b, d) */ * from t1",
"Warnings": [
"[planner:1815]use_index_merge(test.t3, a, b, d) is inapplicable, check whether the table(test.t3) exists"
]
}
]
}
]
2 changes: 2 additions & 0 deletions planner/core/testdata/plan_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,8 @@
"select /*+ USE_INDEX_MERGE(t, primary, f_g) */ * from t where a < 1 or f > 2",
"select /*+ USE_INDEX_MERGE(t, primary, f_g, c_d_e) */ * from t where a < 1 or f > 2",
"select /*+ NO_INDEX_MERGE(), USE_INDEX_MERGE(t, primary, f_g, c_d_e) */ * from t where a < 1 or f > 2",
"select /*+ USE_INDEX_MERGE(t1, c_d_e, f_g) */ * from t where c < 1 or f > 2",
"select /*+ NO_INDEX_MERGE(), USE_INDEX_MERGE(t, primary, f_g, c_d_e) */ * from t where a < 1 or f > 2",
"select /*+ USE_INDEX_MERGE(t) USE_INDEX_MERGE(t) */ * from t where c < 1 or f > 2"
]
},
Expand Down
22 changes: 17 additions & 5 deletions planner/core/testdata/plan_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -102,13 +102,13 @@
{
"SQL": "select /*+ USE_INDEX(t, c_d_e) */ * from t t1",
"Best": "TableReader(Table(t))",
"HasWarn": false,
"HasWarn": true,
"Hints": "USE_INDEX(@`sel_1` `test`.`t1` )"
},
{
"SQL": "select /*+ IGNORE_INDEX(t, c_d_e) */ t1.c from t t1 order by t1.c",
"Best": "IndexReader(Index(t.c_d_e)[[NULL,+inf]])",
"HasWarn": false,
"HasWarn": true,
"Hints": "USE_INDEX(@`sel_1` `test`.`t1` `c_d_e`)"
},
{
Expand Down Expand Up @@ -212,6 +212,18 @@
"HasWarn": true,
"Hints": "USE_INDEX(@`sel_1` `test`.`t` )"
},
{
"SQL": "select /*+ USE_INDEX_MERGE(t1, c_d_e, f_g) */ * from t where c < 1 or f > 2",
"Best": "TableReader(Table(t)->Sel([or(lt(test.t.c, 1), gt(test.t.f, 2))]))",
"HasWarn": true,
"Hints": "USE_INDEX(@`sel_1` `test`.`t` )"
},
{
"SQL": "select /*+ NO_INDEX_MERGE(), USE_INDEX_MERGE(t, primary, f_g, c_d_e) */ * from t where a < 1 or f > 2",
"Best": "TableReader(Table(t)->Sel([or(lt(test.t.a, 1), gt(test.t.f, 2))]))",
"HasWarn": true,
"Hints": "USE_INDEX(@`sel_1` `test`.`t` )"
},
{
"SQL": "select /*+ USE_INDEX_MERGE(t) USE_INDEX_MERGE(t) */ * from t where c < 1 or f > 2",
"Best": "IndexMergeReader(PartialPlans->[Index(t.c_d_e)[[-inf,1)], Index(t.f_g)[(2,+inf]]], TablePlan->Table(t))",
Expand Down Expand Up @@ -1401,17 +1413,17 @@
{
"SQL": "select /*+ AGG_TO_COP(), HASH_AGG(), USE_INDEX(t) */ sum(a) from ta group by a",
"Best": "IndexReader(Index(ta.a)[[NULL,+inf]]->HashAgg)->HashAgg",
"Warning": ""
"Warning": "[planner:1815]use_index(test.t) is inapplicable, check whether the table(test.t) exists"
},
{
"SQL": "select /*+ AGG_TO_COP(), USE_INDEX(t) */ sum(b) from ta group by b",
"Best": "TableReader(Table(ta)->HashAgg)->HashAgg",
"Warning": ""
"Warning": "[planner:1815]use_index(test.t) is inapplicable, check whether the table(test.t) exists"
},
{
"SQL": "select /*+ AGG_TO_COP(), HASH_AGG(), USE_INDEX(t) */ distinct a from ta group by a",
"Best": "IndexReader(Index(ta.a)[[NULL,+inf]]->HashAgg)->HashAgg",
"Warning": ""
"Warning": "[planner:1815]use_index(test.t) is inapplicable, check whether the table(test.t) exists"
},
{
"SQL": "select /*+ AGG_TO_COP(), HASH_AGG(), HASH_JOIN(t1), USE_INDEX(t1), USE_INDEX(t2) */ sum(t1.a) from ta t1, ta t2 where t1.a = t2.b group by t1.a",
Expand Down

0 comments on commit 2cf7a2c

Please sign in to comment.