Skip to content

Commit

Permalink
planner: throw error when create not supported binding from history (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
fzzf678 committed Dec 6, 2022
1 parent 1ddc592 commit 2ca37cb
Show file tree
Hide file tree
Showing 6 changed files with 102 additions and 1 deletion.
22 changes: 22 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1471,3 +1471,25 @@ func TestCreateBindingForPrepareFromHistory(t *testing.T) {
tk.MustExec("execute stmt using @a")
tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1"))
}

func TestErrorCasesCreateBindingFromHistory(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil))

tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2, t3")
tk.MustExec("create table t1(id int)")
tk.MustExec("create table t2(id int)")
tk.MustExec("create table t3(id int)")

sql := "select * from t1 where t1.id in (select id from t2)"
tk.MustExec(sql)
planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows()
tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with sub query")

sql = "select * from t1, t2, t3 where t1.id = t2.id and t2.id = t3.id"
tk.MustExec(sql)
planDigest = tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows()
tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with more than two table join")
}
1 change: 1 addition & 0 deletions executor/tiflashtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ go_test(
"//domain",
"//executor",
"//meta/autoid",
"//parser/auth",
"//parser/terror",
"//store/mockstore",
"//store/mockstore/unistore",
Expand Down
22 changes: 22 additions & 0 deletions executor/tiflashtest/tiflash_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/unistore"
Expand Down Expand Up @@ -1256,3 +1257,24 @@ func TestTiflashEmptyDynamicPruneResult(t *testing.T) {
tk.MustQuery("select /*+ read_from_storage(tiflash[t2]) */ * from IDT_RP24833 partition(p2) t2 where t2. col1 <= -8448770111093677011;").Check(testkit.Rows())
tk.MustQuery("select /*+ read_from_storage(tiflash[t1, t2]) */ * from IDT_RP24833 partition(p3, p4) t1 join IDT_RP24833 partition(p2) t2 on t1.col1 = t2.col1 where t1. col1 between -8448770111093677011 and -8448770111093677011 and t2. col1 <= -8448770111093677011;").Check(testkit.Rows())
}

func TestBindingFromHistoryWithTiFlashBindable(t *testing.T) {
store := testkit.CreateMockStore(t, withMockTiFlash(2))
tk := testkit.NewTestKit(t, store)
require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil))
tk.MustExec("use test;")
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t(a int);")
tk.MustExec("alter table test.t set tiflash replica 1")
tb := external.GetTableByName(t, tk, "test", "t")
err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true)
require.NoError(t, err)
tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'")

sql := "select * from t"
tk.MustExec(sql)
rows := tk.MustQuery("explain select * from t").Rows()
fmt.Println(rows)
planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows()
tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with tiflash engine")
}
4 changes: 3 additions & 1 deletion planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1054,7 +1054,9 @@ func (b *PlanBuilder) buildCreateBindPlanFromPlanDigest(v *ast.CreateBindingStmt
if err != nil {
return nil, errors.Errorf("binding failed: %v", err)
}

if err = hint.CheckBindingFromHistoryBindable(originNode, bindableStmt.PlanHint); err != nil {
return nil, err
}
bindSQL := bindinfo.GenerateBindSQL(context.TODO(), originNode, bindableStmt.PlanHint, true, bindableStmt.Schema)
var hintNode ast.StmtNode
hintNode, err = parser4binding.ParseOneStmt(bindSQL, bindableStmt.Charset, bindableStmt.Collation)
Expand Down
1 change: 1 addition & 0 deletions util/hint/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ go_library(
"//sessionctx",
"//util/dbterror",
"//util/logutil",
"@com_github_pingcap_errors//:errors",
"@org_uber_go_zap//:zap",
],
)
53 changes: 53 additions & 0 deletions util/hint/hint_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"strconv"
"strings"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
Expand Down Expand Up @@ -617,3 +618,55 @@ func GenerateQBName(nodeType NodeType, blockOffset int) (model.CIStr, error) {
}
return model.NewCIStr(fmt.Sprintf("%s%d", defaultSelectBlockPrefix, blockOffset)), nil
}

// CheckBindingFromHistoryBindable checks whether the ast and hint string from history is bindable.
// Not support:
// 1. query use tiFlash engine
// 2. query with sub query
// 3. query with more than 2 table join
func CheckBindingFromHistoryBindable(node ast.Node, hintStr string) error {
// check tiflash
contain := strings.Contains(hintStr, "tiflash")
if contain {
return errors.New("can't create binding for query with tiflash engine")
}

checker := bindableChecker{
bindable: true,
tables: make(map[model.CIStr]struct{}, 2),
}
node.Accept(&checker)
return checker.reason
}

// bindableChecker checks whether a binding from history can be created.
type bindableChecker struct {
bindable bool
reason error
tables map[model.CIStr]struct{}
}

// Enter implements Visitor interface.
func (checker *bindableChecker) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
switch node := in.(type) {
case *ast.ExistsSubqueryExpr, *ast.SubqueryExpr:
checker.bindable = false
checker.reason = errors.New("can't create binding for query with sub query")
return in, true
case *ast.TableName:
if _, ok := checker.tables[node.Schema]; !ok {
checker.tables[node.Name] = struct{}{}
}
if len(checker.tables) >= 3 {
checker.bindable = false
checker.reason = errors.New("can't create binding for query with more than two table join")
return in, true
}
}
return in, false
}

// Leave implements Visitor interface.
func (checker *bindableChecker) Leave(in ast.Node) (out ast.Node, ok bool) {
return in, checker.bindable
}

0 comments on commit 2ca37cb

Please sign in to comment.