Skip to content

Commit

Permalink
planner: optimize the performance of non-cacheable-checker (#42002)
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored Mar 8, 2023
1 parent 31e4894 commit 4ac7dc9
Show file tree
Hide file tree
Showing 2 changed files with 52 additions and 7 deletions.
31 changes: 24 additions & 7 deletions planner/core/plan_cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package core

import (
"fmt"
"sync"

"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
Expand Down Expand Up @@ -201,6 +202,8 @@ func NonPreparedPlanCacheable(node ast.Node, is infoschema.InfoSchema) bool {
return ok
}

var nonPrepCacheCheckerPool = &sync.Pool{New: func() any { return &nonPreparedPlanCacheableChecker{} }}

// NonPreparedPlanCacheableWithCtx checks whether the input ast is cacheable for non-prepared plan cache.
// Only support: select {field} from {single-table} where {cond} and {cond} ...
// {cond}: {col} {op} {val}
Expand Down Expand Up @@ -238,13 +241,17 @@ func NonPreparedPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is
}
}

checker := nonPreparedPlanCacheableChecker{
sctx: sctx,
cacheable: true,
schema: is,
}
node.Accept(&checker)
return checker.cacheable, checker.reason
// allocate and init the checker
checker := nonPrepCacheCheckerPool.Get().(*nonPreparedPlanCacheableChecker)
checker.reset(sctx, is)

node.Accept(checker)
cacheable, reason := checker.cacheable, checker.reason

// put the checker back
nonPrepCacheCheckerPool.Put(checker)

return cacheable, reason
}

// nonPreparedPlanCacheableChecker checks whether a query's plan can be cached for non-prepared plan cache.
Expand All @@ -260,6 +267,16 @@ type nonPreparedPlanCacheableChecker struct {
filterCnt int // the number of filters in the current node
}

func (checker *nonPreparedPlanCacheableChecker) reset(sctx sessionctx.Context, schema infoschema.InfoSchema) {
checker.sctx = sctx
checker.cacheable = true
checker.schema = schema
checker.reason = ""
checker.tableNode = nil
checker.constCnt = 0
checker.filterCnt = 0
}

// Enter implements Visitor interface.
func (checker *nonPreparedPlanCacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
if checker.isFilterNode(in) {
Expand Down
28 changes: 28 additions & 0 deletions planner/core/plan_cacheable_checker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessiontxn"
"github.com/pingcap/tidb/testkit"
driver "github.com/pingcap/tidb/types/parser_driver"
"github.com/pingcap/tidb/util/mock"
Expand Down Expand Up @@ -333,3 +334,30 @@ func TestNonPreparedPlanCacheable(t *testing.T) {
require.True(t, core.NonPreparedPlanCacheable(stmt, is))
}
}

func BenchmarkNonPreparedPlanCacheableChecker(b *testing.B) {
store := testkit.CreateMockStore(b)
tk := testkit.NewTestKit(b, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int)")

p := parser.New()
sql := "select * from test.t where a<10"
stmt, err := p.ParseOneStmt(sql, "", "")
if err != nil {
b.Fatal(err)
}
sctx := tk.Session()
is := sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema()

core.NonPreparedPlanCacheable(stmt, is)

b.ResetTimer()
for i := 0; i < b.N; i++ {
ok, _ := core.NonPreparedPlanCacheableWithCtx(sctx, stmt, is)
if !ok {
b.Fatal()
}
}
}

0 comments on commit 4ac7dc9

Please sign in to comment.