Skip to content

Commit

Permalink
Merge branch 'master' into fix#11197
Browse files Browse the repository at this point in the history
  • Loading branch information
Reminiscent authored Aug 26, 2019
2 parents d5e5458 + f24866a commit 86b9327
Show file tree
Hide file tree
Showing 67 changed files with 1,927 additions and 856 deletions.
177 changes: 38 additions & 139 deletions bindinfo/bind.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,155 +15,54 @@ package bindinfo

import "github.com/pingcap/parser/ast"

// BindHint will add hints for originStmt according to hintedStmt' hints.
func BindHint(originStmt, hintedStmt ast.StmtNode) ast.StmtNode {
switch x := originStmt.(type) {
case *ast.SelectStmt:
return selectBind(x, hintedStmt.(*ast.SelectStmt))
default:
return originStmt
}
}

func selectBind(originalNode, hintedNode *ast.SelectStmt) *ast.SelectStmt {
if hintedNode.TableHints != nil {
originalNode.TableHints = hintedNode.TableHints
}
if originalNode.From != nil {
originalNode.From.TableRefs = resultSetNodeBind(originalNode.From.TableRefs, hintedNode.From.TableRefs).(*ast.Join)
}
if originalNode.Where != nil {
originalNode.Where = exprBind(originalNode.Where, hintedNode.Where).(ast.ExprNode)
}

if originalNode.Having != nil {
originalNode.Having.Expr = exprBind(originalNode.Having.Expr, hintedNode.Having.Expr)
}

if originalNode.OrderBy != nil {
originalNode.OrderBy = orderByBind(originalNode.OrderBy, hintedNode.OrderBy)
}

if originalNode.Fields != nil {
origFields := originalNode.Fields.Fields
hintFields := hintedNode.Fields.Fields
for idx := range origFields {
origFields[idx].Expr = exprBind(origFields[idx].Expr, hintFields[idx].Expr)
}
}
return originalNode
// HintsSet contains all hints of a query.
type HintsSet struct {
tableHints [][]*ast.TableOptimizerHint // Slice offset is the traversal order of `SelectStmt` in the ast.
indexHints [][]*ast.IndexHint // Slice offset is the traversal order of `TableName` in the ast.
}

func orderByBind(originalNode, hintedNode *ast.OrderByClause) *ast.OrderByClause {
for idx := 0; idx < len(originalNode.Items); idx++ {
originalNode.Items[idx].Expr = exprBind(originalNode.Items[idx].Expr, hintedNode.Items[idx].Expr)
}
return originalNode
type hintProcessor struct {
*HintsSet
// bindHint2Ast indicates the behavior of the processor, `true` for bind hint to ast, `false` for extract hint from ast.
bindHint2Ast bool
tableCounter int64
indexCounter int64
}

func exprBind(originalNode, hintedNode ast.ExprNode) ast.ExprNode {
switch v := originalNode.(type) {
case *ast.SubqueryExpr:
if v.Query != nil {
v.Query = resultSetNodeBind(v.Query, hintedNode.(*ast.SubqueryExpr).Query)
}
case *ast.ExistsSubqueryExpr:
if v.Sel != nil {
v.Sel.(*ast.SubqueryExpr).Query = resultSetNodeBind(v.Sel.(*ast.SubqueryExpr).Query, hintedNode.(*ast.ExistsSubqueryExpr).Sel.(*ast.SubqueryExpr).Query)
}
case *ast.PatternInExpr:
if v.Sel != nil {
v.Sel.(*ast.SubqueryExpr).Query = resultSetNodeBind(v.Sel.(*ast.SubqueryExpr).Query, hintedNode.(*ast.PatternInExpr).Sel.(*ast.SubqueryExpr).Query)
}
case *ast.BinaryOperationExpr:
if v.L != nil {
v.L = exprBind(v.L, hintedNode.(*ast.BinaryOperationExpr).L)
}
if v.R != nil {
v.R = exprBind(v.R, hintedNode.(*ast.BinaryOperationExpr).R)
}
case *ast.IsNullExpr:
if v.Expr != nil {
v.Expr = exprBind(v.Expr, hintedNode.(*ast.IsNullExpr).Expr)
}
case *ast.IsTruthExpr:
if v.Expr != nil {
v.Expr = exprBind(v.Expr, hintedNode.(*ast.IsTruthExpr).Expr)
}
case *ast.PatternLikeExpr:
if v.Pattern != nil {
v.Pattern = exprBind(v.Pattern, hintedNode.(*ast.PatternLikeExpr).Pattern)
}
case *ast.CompareSubqueryExpr:
if v.L != nil {
v.L = exprBind(v.L, hintedNode.(*ast.CompareSubqueryExpr).L)
}
if v.R != nil {
v.R = exprBind(v.R, hintedNode.(*ast.CompareSubqueryExpr).R)
}
case *ast.BetweenExpr:
if v.Left != nil {
v.Left = exprBind(v.Left, hintedNode.(*ast.BetweenExpr).Left)
}
if v.Right != nil {
v.Right = exprBind(v.Right, hintedNode.(*ast.BetweenExpr).Right)
}
case *ast.UnaryOperationExpr:
if v.V != nil {
v.V = exprBind(v.V, hintedNode.(*ast.UnaryOperationExpr).V)
}
case *ast.CaseExpr:
if v.Value != nil {
v.Value = exprBind(v.Value, hintedNode.(*ast.CaseExpr).Value)
}
if v.ElseClause != nil {
v.ElseClause = exprBind(v.ElseClause, hintedNode.(*ast.CaseExpr).ElseClause)
func (hp *hintProcessor) Enter(in ast.Node) (ast.Node, bool) {
switch v := in.(type) {
case *ast.SelectStmt:
if hp.bindHint2Ast {
v.TableHints = hp.tableHints[hp.tableCounter]
hp.tableCounter++
} else {
hp.tableHints = append(hp.tableHints, v.TableHints)
}
case *ast.TableName:
if hp.bindHint2Ast {
v.IndexHints = hp.indexHints[hp.indexCounter]
hp.indexCounter++
} else {
hp.indexHints = append(hp.indexHints, v.IndexHints)
}
}
return originalNode
return in, false
}

func resultSetNodeBind(originalNode, hintedNode ast.ResultSetNode) ast.ResultSetNode {
switch x := originalNode.(type) {
case *ast.Join:
return joinBind(x, hintedNode.(*ast.Join))
case *ast.TableSource:
ts, _ := hintedNode.(*ast.TableSource)
switch v := x.Source.(type) {
case *ast.SelectStmt:
x.Source = selectBind(v, ts.Source.(*ast.SelectStmt))
case *ast.UnionStmt:
x.Source = unionSelectBind(v, hintedNode.(*ast.TableSource).Source.(*ast.UnionStmt))
case *ast.TableName:
x.Source.(*ast.TableName).IndexHints = ts.Source.(*ast.TableName).IndexHints
}
return x
case *ast.SelectStmt:
return selectBind(x, hintedNode.(*ast.SelectStmt))
case *ast.UnionStmt:
return unionSelectBind(x, hintedNode.(*ast.UnionStmt))
default:
return x
}
func (hp *hintProcessor) Leave(in ast.Node) (ast.Node, bool) {
return in, true
}

func joinBind(originalNode, hintedNode *ast.Join) *ast.Join {
if originalNode.Left != nil {
originalNode.Left = resultSetNodeBind(originalNode.Left, hintedNode.Left)
}

if hintedNode.Right != nil {
originalNode.Right = resultSetNodeBind(originalNode.Right, hintedNode.Right)
}

return originalNode
// CollectHint collects hints for a statement.
func CollectHint(in ast.StmtNode) *HintsSet {
hp := hintProcessor{HintsSet: &HintsSet{tableHints: make([][]*ast.TableOptimizerHint, 0, 4), indexHints: make([][]*ast.IndexHint, 0, 4)}}
in.Accept(&hp)
return hp.HintsSet
}

func unionSelectBind(originalNode, hintedNode *ast.UnionStmt) ast.ResultSetNode {
selects := originalNode.SelectList.Selects
for i := len(selects) - 1; i >= 0; i-- {
originalNode.SelectList.Selects[i] = selectBind(selects[i], hintedNode.SelectList.Selects[i])
}

return originalNode
// BindHint will add hints for stmt according to the hints in `hintsSet`.
func BindHint(stmt ast.StmtNode, hintsSet *HintsSet) ast.StmtNode {
hp := hintProcessor{HintsSet: hintsSet, bindHint2Ast: true}
stmt.Accept(&hp)
return stmt
}
20 changes: 20 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -466,3 +466,23 @@ func (s *testSuite) TestErrorBind(c *C) {
c.Check(err, IsNil)
c.Check(chk.NumRows(), Equals, 0)
}

func (s *testSuite) TestPreparedStmt(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, index idx(a))")
tk.MustExec(`prepare stmt1 from 'select * from t'`)
tk.MustExec("execute stmt1")
c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 0)

tk.MustExec("create binding for select * from t using select * from t use index(idx)")
tk.MustExec("execute stmt1")
c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 1)
c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx")

tk.MustExec("drop binding for select * from t")
tk.MustExec("execute stmt1")
c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 0)
}
4 changes: 2 additions & 2 deletions bindinfo/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ package bindinfo
import (
"unsafe"

"github.com/pingcap/parser/ast"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
Expand All @@ -34,7 +33,8 @@ const (
// BindMeta stores the basic bind info and bindSql astNode.
type BindMeta struct {
*BindRecord
Ast ast.StmtNode //ast will be used to do query sql bind check
// HintSet stores the set of hints of binding sql.
*HintsSet
}

// cache is a k-v map, key is original sql, value is a slice of BindMeta.
Expand Down
6 changes: 3 additions & 3 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,7 +218,7 @@ func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) {
return
}

hash, meta := newBindMetaWithoutAst(record)
hash, meta := newBindMetaWithoutHints(record)
h.removeBindMeta(hash, meta)
}()

Expand Down Expand Up @@ -308,11 +308,11 @@ func (h *BindHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMet
if err != nil {
return "", nil, err
}
meta = &BindMeta{BindRecord: record, Ast: stmtNodes[0]}
meta = &BindMeta{BindRecord: record, HintsSet: CollectHint(stmtNodes[0])}
return hash, meta, nil
}

func newBindMetaWithoutAst(record *BindRecord) (hash string, meta *BindMeta) {
func newBindMetaWithoutHints(record *BindRecord) (hash string, meta *BindMeta) {
hash = parser.DigestHash(record.OriginalSQL)
meta = &BindMeta{BindRecord: record}
return hash, meta
Expand Down
2 changes: 1 addition & 1 deletion bindinfo/session_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ func (h *SessionHandle) newBindMeta(record *BindRecord) (hash string, meta *Bind
if err != nil {
return "", nil, err
}
meta = &BindMeta{BindRecord: record, Ast: stmtNodes[0]}
meta = &BindMeta{BindRecord: record, HintsSet: CollectHint(stmtNodes[0])}
return hash, meta, nil
}

Expand Down
4 changes: 2 additions & 2 deletions cmd/explaintest/r/explain_complex.result
Original file line number Diff line number Diff line change
Expand Up @@ -107,10 +107,10 @@ explain SELECT `ds`, `p1`, `p2`, `p3`, `p4`, `p5`, `p6_md5`, `p7_md5`, count(dic
id count task operator info
Projection_7 53.00 root test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, install_device
└─Sort_8 53.00 root test.dt.ds2:desc
└─HashAgg_16 53.00 root group by:col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, funcs:count(col_0), firstrow(col_1), firstrow(col_2), firstrow(col_3), firstrow(col_4), firstrow(col_5), firstrow(col_6), firstrow(col_7), firstrow(col_8), firstrow(col_9)
└─HashAgg_16 53.00 root group by:col_10, col_11, col_12, col_13, col_14, col_15, col_16, col_17, funcs:count(col_0), firstrow(col_10), firstrow(col_2), firstrow(col_11), firstrow(col_12), firstrow(col_13), firstrow(col_14), firstrow(col_15), firstrow(col_16), firstrow(col_17)
└─IndexLookUp_17 53.00 root
├─IndexScan_13 2650.00 cop table:dt, index:cm, range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false, stats:pseudo
└─HashAgg_11 53.00 cop group by:test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, funcs:count(test.dt.dic), firstrow(test.dt.ds), firstrow(test.dt.ds2), firstrow(test.dt.p1), firstrow(test.dt.p2), firstrow(test.dt.p3), firstrow(test.dt.p4), firstrow(test.dt.p5), firstrow(test.dt.p6_md5), firstrow(test.dt.p7_md5)
└─HashAgg_11 53.00 cop group by:test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5, funcs:count(test.dt.dic), firstrow(test.dt.ds2)
└─Selection_15 66.25 cop ge(test.dt.ds, 2016-09-01 00:00:00.000000), le(test.dt.ds, 2016-11-03 00:00:00.000000)
└─TableScan_14 2650.00 cop table:dt, keep order:false, stats:pseudo
explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext, gad.t as gtime from st gad join (select id, aid, pt, dic, ip, t from dd where pt = 'android' and bm = 0 and t > 1478143908) sdk on gad.aid = sdk.aid and gad.ip = sdk.ip and sdk.t > gad.t where gad.t > 1478143908 and gad.pt = 'android' group by gad.aid, sdk.dic limit 2500;
Expand Down
Loading

0 comments on commit 86b9327

Please sign in to comment.