From 27069dc4565191b8ecfb6fcc5a84ed3284aaa0c5 Mon Sep 17 00:00:00 2001 From: imtbkcat Date: Tue, 14 Apr 2020 21:48:38 +0800 Subject: [PATCH 1/4] let partition table be not cacheable --- executor/prepared.go | 2 +- planner/core/cacheable_checker.go | 18 +++- planner/core/cacheable_checker_test.go | 110 +++++++++++++++++-------- planner/core/common_plans.go | 37 ++------- server.toml | 4 + 5 files changed, 101 insertions(+), 70 deletions(-) create mode 100644 server.toml diff --git a/executor/prepared.go b/executor/prepared.go index 102f3f4a3f883..de869a875dba8 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -178,7 +178,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { SchemaVersion: e.is.SchemaMetaVersion(), } - prepared.UseCache = plannercore.PreparedPlanCacheEnabled() && plannercore.Cacheable(stmt) + prepared.UseCache = plannercore.PreparedPlanCacheEnabled() && plannercore.Cacheable(stmt, e.is) // We try to build the real statement of preparedStmt. for i := range prepared.Params { diff --git a/planner/core/cacheable_checker.go b/planner/core/cacheable_checker.go index 7369c9ac1cfec..d57fa7e1ee56a 100644 --- a/planner/core/cacheable_checker.go +++ b/planner/core/cacheable_checker.go @@ -16,13 +16,14 @@ package core import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/types/parser_driver" ) // Cacheable checks whether the input ast is cacheable. // Handle "ignore_plan_cache()" hint // If there are multiple hints, only one will take effect -func Cacheable(node ast.Node) bool { +func Cacheable(node ast.Node, is infoschema.InfoSchema) bool { switch node.(type) { case *ast.SelectStmt: for _, hints := range (node.(*ast.SelectStmt)).TableHints { @@ -48,6 +49,7 @@ func Cacheable(node ast.Node) bool { } checker := cacheableChecker{ cacheable: true, + schema: is, } node.Accept(&checker) return checker.cacheable @@ -60,6 +62,7 @@ func Cacheable(node ast.Node) bool { // NOTE: we can add more rules in the future. type cacheableChecker struct { cacheable bool + schema infoschema.InfoSchema } // Enter implements Visitor interface. @@ -105,10 +108,23 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren checker.cacheable = false return in, true } + case *ast.TableName: + if checker.isPartitionTable(node) { + checker.cacheable = false + return in, true + } } return in, false } +func (checker *cacheableChecker) isPartitionTable(tn *ast.TableName) bool { + tb, _ := checker.schema.TableByName(tn.Schema, tn.Name) + if tb.Meta().Partition != nil { + return true + } + return false +} + // Leave implements Visitor interface. func (checker *cacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) { return in, checker.cacheable diff --git a/planner/core/cacheable_checker_test.go b/planner/core/cacheable_checker_test.go index d2db2d42392a9..97370f72cdf91 100644 --- a/planner/core/cacheable_checker_test.go +++ b/planner/core/cacheable_checker_test.go @@ -11,14 +11,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core +package core_test import ( . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/util/testkit" ) var _ = Suite(&testCacheableSuite{}) @@ -27,20 +30,33 @@ type testCacheableSuite struct { } func (s *testCacheableSuite) TestCacheable(c *C) { + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + defer func() { + dom.Close() + store.Close() + }() + tk.MustExec("use test") + tk.MustExec("create table t1(a int, b int) partition by range(a) ( partition p0 values less than (6), partition p1 values less than (11) )") + tk.MustExec("create table t2(a int, b int) partition by hash(a) partitions 11") + tk.MustExec("create table t3(a int, b int)") + tbl := &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t3")} + is := infoschema.GetInfoSchema(tk.Se) // test non-SelectStmt/-InsertStmt/-DeleteStmt/-UpdateStmt/-SelectStmt var stmt ast.Node = &ast.UnionStmt{} - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) stmt = &ast.ShowStmt{} - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) stmt = &ast.LoadDataStmt{} - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) - tableRefsClause := &ast.TableRefsClause{TableRefs: &ast.Join{Left: &ast.TableSource{Source: &ast.TableName{}}}} + tableRefsClause := &ast.TableRefsClause{TableRefs: &ast.Join{Left: &ast.TableSource{Source: tbl}}} // test InsertStmt stmt = &ast.InsertStmt{Table: tableRefsClause} - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) // test DeleteStmt whereExpr := &ast.FuncCallExpr{} @@ -48,21 +64,21 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Where: whereExpr, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) for funcName := range expression.UnCacheableFunctions { whereExpr.FnName = model.NewCIStr(funcName) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) } whereExpr.FnName = model.NewCIStr(ast.Rand) - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt = &ast.DeleteStmt{ TableRefs: tableRefsClause, Where: &ast.ExistsSubqueryExpr{}, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt := &ast.Limit{ Count: &driver.ParamMarkerExpr{}, @@ -71,7 +87,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{ Offset: &driver.ParamMarkerExpr{}, @@ -80,19 +96,19 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{} stmt = &ast.DeleteStmt{ TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt.(*ast.DeleteStmt).TableHints = append(stmt.(*ast.DeleteStmt).TableHints, &ast.TableOptimizerHint{ - HintName: model.NewCIStr(HintIgnorePlanCache), + HintName: model.NewCIStr(core.HintIgnorePlanCache), }) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) // test UpdateStmt whereExpr = &ast.FuncCallExpr{} @@ -100,21 +116,21 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Where: whereExpr, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) for funcName := range expression.UnCacheableFunctions { whereExpr.FnName = model.NewCIStr(funcName) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) } whereExpr.FnName = model.NewCIStr(ast.Rand) - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt = &ast.UpdateStmt{ TableRefs: tableRefsClause, Where: &ast.ExistsSubqueryExpr{}, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{ Count: &driver.ParamMarkerExpr{}, @@ -123,7 +139,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{ Offset: &driver.ParamMarkerExpr{}, @@ -132,39 +148,39 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{} stmt = &ast.UpdateStmt{ TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt.(*ast.UpdateStmt).TableHints = append(stmt.(*ast.UpdateStmt).TableHints, &ast.TableOptimizerHint{ - HintName: model.NewCIStr(HintIgnorePlanCache), + HintName: model.NewCIStr(core.HintIgnorePlanCache), }) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) // test SelectStmt whereExpr = &ast.FuncCallExpr{} stmt = &ast.SelectStmt{ Where: whereExpr, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) for funcName := range expression.UnCacheableFunctions { whereExpr.FnName = model.NewCIStr(funcName) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) } whereExpr.FnName = model.NewCIStr(ast.Rand) - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt = &ast.SelectStmt{ Where: &ast.ExistsSubqueryExpr{}, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{ Count: &driver.ParamMarkerExpr{}, @@ -172,7 +188,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { stmt = &ast.SelectStmt{ Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{ Offset: &driver.ParamMarkerExpr{}, @@ -180,33 +196,55 @@ func (s *testCacheableSuite) TestCacheable(c *C) { stmt = &ast.SelectStmt{ Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{} stmt = &ast.SelectStmt{ Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) paramExpr := &driver.ParamMarkerExpr{} orderByClause := &ast.OrderByClause{Items: []*ast.ByItem{{Expr: paramExpr}}} stmt = &ast.SelectStmt{ OrderBy: orderByClause, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) valExpr := &driver.ValueExpr{} orderByClause = &ast.OrderByClause{Items: []*ast.ByItem{{Expr: valExpr}}} stmt = &ast.SelectStmt{ OrderBy: orderByClause, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt.(*ast.SelectStmt).TableHints = append(stmt.(*ast.SelectStmt).TableHints, &ast.TableOptimizerHint{ - HintName: model.NewCIStr(HintIgnorePlanCache), + HintName: model.NewCIStr(core.HintIgnorePlanCache), }) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) boundExpr := &ast.FrameBound{Expr: &driver.ParamMarkerExpr{}} - c.Assert(Cacheable(boundExpr), IsFalse) + c.Assert(core.Cacheable(boundExpr, is), IsFalse) + + // Partition table can not be cached. + join := &ast.Join{ + Left: &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t1")}, + Right: &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t2")}, + } + stmt = &ast.SelectStmt{ + From: &ast.TableRefsClause{ + TableRefs: join, + }, + } + c.Assert(core.Cacheable(stmt, is), IsFalse) + + join = &ast.Join{ + Left: &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t3")}, + } + stmt = &ast.SelectStmt{ + From: &ast.TableRefsClause{ + TableRefs: join, + }, + } + c.Assert(core.Cacheable(stmt, is), IsTrue) } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index fa15447965e98..10f11643c52cf 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -348,9 +348,12 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, } e.names = names e.Plan = p - isRange := e.isRangePartition(p) _, isTableDual := p.(*PhysicalTableDual) - if !isTableDual && prepared.UseCache && !isRange { + if !isTableDual && prepared.UseCache { + err = e.setFoundInPlanCache(sctx, true) + if err != nil { + return err + } cached := NewPSTMTPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan) preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) @@ -526,36 +529,6 @@ func (e *Execute) rebuildRange(p Plan) error { return nil } -func checkRangePartitionInfo(pi *model.PartitionInfo) bool { - if pi != nil && pi.Type == model.PartitionTypeRange { - return true - } - return false -} - -// Prepare plan cache is not support query plan on range partition table. -func (e *Execute) isRangePartition(p Plan) bool { - isRange := false - switch x := p.(type) { - case *PhysicalTableReader: - ts := x.TablePlans[0].(*PhysicalTableScan) - return checkRangePartitionInfo(ts.Table.Partition) - case *PhysicalIndexLookUpReader: - is := x.IndexPlans[0].(*PhysicalIndexScan) - return checkRangePartitionInfo(is.Table.Partition) - case *PhysicalIndexReader: - is := x.IndexPlans[0].(*PhysicalIndexScan) - return checkRangePartitionInfo(is.Table.Partition) - case PhysicalPlan: - for _, child := range x.Children() { - if e.isRangePartition(child) { - isRange = true - } - } - } - return isRange -} - func (e *Execute) buildRangeForIndexScan(sctx sessionctx.Context, is *PhysicalIndexScan) ([]*ranger.Range, error) { if len(is.IdxCols) == 0 { return ranger.FullRange(), nil diff --git a/server.toml b/server.toml new file mode 100644 index 0000000000000..0f2a1653310b4 --- /dev/null +++ b/server.toml @@ -0,0 +1,4 @@ +[prepared-plan-cache] +enabled = true +capacity = 100 +memory-guard-ratio = 0.1 From ec3b479a2067f0cd9ebcb76c19e085ba57f977d2 Mon Sep 17 00:00:00 2001 From: imtbkcat Date: Tue, 14 Apr 2020 22:03:06 +0800 Subject: [PATCH 2/4] remove testfile --- server.toml | 4 ---- 1 file changed, 4 deletions(-) delete mode 100644 server.toml diff --git a/server.toml b/server.toml deleted file mode 100644 index 0f2a1653310b4..0000000000000 --- a/server.toml +++ /dev/null @@ -1,4 +0,0 @@ -[prepared-plan-cache] -enabled = true -capacity = 100 -memory-guard-ratio = 0.1 From 135e8966c9bbe76a636ad63972c348d26e321541 Mon Sep 17 00:00:00 2001 From: imtbkcat Date: Tue, 14 Apr 2020 22:51:45 +0800 Subject: [PATCH 3/4] donot ignore error --- planner/core/cacheable_checker.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/planner/core/cacheable_checker.go b/planner/core/cacheable_checker.go index d57fa7e1ee56a..7b4b740986c8f 100644 --- a/planner/core/cacheable_checker.go +++ b/planner/core/cacheable_checker.go @@ -18,6 +18,8 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" ) // Cacheable checks whether the input ast is cacheable. @@ -118,7 +120,11 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren } func (checker *cacheableChecker) isPartitionTable(tn *ast.TableName) bool { - tb, _ := checker.schema.TableByName(tn.Schema, tn.Name) + tb, err := checker.schema.TableByName(tn.Schema, tn.Name) + if err != nil { + return false + logutil.BgLogger().Error("Error occur in checking cacheable", zap.Error(err)) + } if tb.Meta().Partition != nil { return true } From 99f4ba175ccf77ce73eb939b8a7a81f8ddb8b954 Mon Sep 17 00:00:00 2001 From: imtbkcat Date: Tue, 14 Apr 2020 22:57:37 +0800 Subject: [PATCH 4/4] fix ci --- planner/core/cacheable_checker.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/cacheable_checker.go b/planner/core/cacheable_checker.go index 7b4b740986c8f..d6c3161417d11 100644 --- a/planner/core/cacheable_checker.go +++ b/planner/core/cacheable_checker.go @@ -122,8 +122,8 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren func (checker *cacheableChecker) isPartitionTable(tn *ast.TableName) bool { tb, err := checker.schema.TableByName(tn.Schema, tn.Name) if err != nil { - return false logutil.BgLogger().Error("Error occur in checking cacheable", zap.Error(err)) + return false } if tb.Meta().Partition != nil { return true