diff --git a/executor/compiler.go b/executor/compiler.go index bb593ec654970..f1cd5af86b3fc 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -90,7 +90,7 @@ func (c *Compiler) compile(ctx context.Context, stmtNode ast.StmtNode, skipBind InfoSchema: infoSchema, Plan: finalPlan, LowerPriority: lowerPriority, - Cacheable: plannercore.Cacheable(stmtNode), + Cacheable: plannercore.Cacheable(stmtNode, infoSchema), Text: stmtNode.Text(), StmtNode: stmtNode, Ctx: c.Ctx, diff --git a/executor/prepared.go b/executor/prepared.go index 20c1d6321ef48..0311a9b40ddab 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -173,7 +173,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { Params: sorter.markers, SchemaVersion: e.is.SchemaMetaVersion(), } - prepared.UseCache = plannercore.PreparedPlanCacheEnabled() && (vars.LightningMode || plannercore.Cacheable(stmt)) + prepared.UseCache = plannercore.PreparedPlanCacheEnabled() && (vars.LightningMode || 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 f4ef4f9c22b6d..283a7316081e0 100644 --- a/planner/core/cacheable_checker.go +++ b/planner/core/cacheable_checker.go @@ -14,13 +14,18 @@ package core import ( + "context" + "github.com/pingcap/parser/ast" "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. -func Cacheable(node ast.Node) bool { +func Cacheable(node ast.Node, is infoschema.InfoSchema) bool { _, isSelect := node.(*ast.SelectStmt) _, isUpdate := node.(*ast.UpdateStmt) _, isInsert := node.(*ast.InsertStmt) @@ -30,6 +35,7 @@ func Cacheable(node ast.Node) bool { } checker := cacheableChecker{ cacheable: true, + schema: is, } node.Accept(&checker) return checker.cacheable @@ -42,6 +48,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. @@ -87,10 +94,27 @@ 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, err := checker.schema.TableByName(tn.Schema, tn.Name) + if err != nil { + logutil.Logger(context.Background()).Error("Error occur in checking cacheable", zap.Error(err)) + return false + } + 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 6d195c25f4b27..40e1c6075e24f 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,14 +96,14 @@ 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) // test UpdateStmt whereExpr = &ast.FuncCallExpr{} @@ -95,21 +111,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{}, @@ -118,7 +134,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{}, @@ -127,34 +143,34 @@ 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) // 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{}, @@ -162,7 +178,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{}, @@ -170,28 +186,50 @@ 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) 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 4283cf0a26665..ed536f5bae2c1 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -253,8 +253,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, return nil, err } _, isTableDual := p.(*PhysicalTableDual) - isPartition := e.isPartition(p) - if !isTableDual && prepared.UseCache && !isPartition { + if !isTableDual && prepared.UseCache { sctx.PreparedPlanCache().Put(cacheKey, NewPSTMTPlanCacheValue(p)) } return p, err @@ -330,36 +329,6 @@ func (e *Execute) rebuildRange(p Plan) error { return nil } -func checkPartitionInfo(pi *model.PartitionInfo) bool { - if pi != nil { - return true - } - return false -} - -// Prepare plan cache is not support query plan on range partition table. -func (e *Execute) isPartition(p Plan) bool { - isRange := false - switch x := p.(type) { - case *PhysicalTableReader: - ts := x.TablePlans[0].(*PhysicalTableScan) - return checkPartitionInfo(ts.Table.Partition) - case *PhysicalIndexLookUpReader: - is := x.IndexPlans[0].(*PhysicalIndexScan) - return checkPartitionInfo(is.Table.Partition) - case *PhysicalIndexReader: - is := x.IndexPlans[0].(*PhysicalIndexScan) - return checkPartitionInfo(is.Table.Partition) - case PhysicalPlan: - for _, child := range x.Children() { - if e.isPartition(child) { - isRange = true - } - } - } - return isRange -} - func (e *Execute) buildRangeForIndexScan(sctx sessionctx.Context, is *PhysicalIndexScan) ([]*ranger.Range, error) { idxCols, colLengths := expression.IndexInfo2Cols(is.schema.Columns, is.Index) if len(idxCols) == 0 {