diff --git a/ddl/metadatalocktest/mdl_test.go b/ddl/metadatalocktest/mdl_test.go index 4a4cc2a90c9a8..334da49010138 100644 --- a/ddl/metadatalocktest/mdl_test.go +++ b/ddl/metadatalocktest/mdl_test.go @@ -909,7 +909,11 @@ func TestMDLPreparePlanCacheExecute(t *testing.T) { tk.MustQuery("select * from t2") tk.MustExec(`set @a = 2, @b=4;`) - tk.MustExec(`execute stmt_test_1 using @a, @b;`) + tk.MustExec(`execute stmt_test_1 using @a, @b;`) // can't reuse the prior plan created outside this txn. + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + tk.MustExec(`execute stmt_test_1 using @a, @b;`) // can't reuse the prior plan since this table becomes dirty. + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + tk.MustExec(`execute stmt_test_1 using @a, @b;`) // can't reuse the prior plan now. tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) // The plan is from cache, the metadata lock should be added to block the DDL. ch <- struct{}{} diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go new file mode 100644 index 0000000000000..71d78db328714 --- /dev/null +++ b/pkg/planner/core/plan_cache.go @@ -0,0 +1,358 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "context" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/metrics" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/planner/core/base" + core_metrics "github.com/pingcap/tidb/pkg/planner/core/metrics" + "github.com/pingcap/tidb/pkg/planner/util/debugtrace" + "github.com/pingcap/tidb/pkg/privilege" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessiontxn/staleread" + "github.com/pingcap/tidb/pkg/types" + driver "github.com/pingcap/tidb/pkg/types/parser_driver" + "github.com/pingcap/tidb/pkg/util/chunk" + contextutil "github.com/pingcap/tidb/pkg/util/context" + "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" +) + +// PlanCacheKeyTestIssue43667 is only for test. +type PlanCacheKeyTestIssue43667 struct{} + +// PlanCacheKeyTestIssue46760 is only for test. +type PlanCacheKeyTestIssue46760 struct{} + +// PlanCacheKeyTestIssue47133 is only for test. +type PlanCacheKeyTestIssue47133 struct{} + +// PlanCacheKeyTestClone is only for test. +type PlanCacheKeyTestClone struct{} + +// SetParameterValuesIntoSCtx sets these parameters into session context. +func SetParameterValuesIntoSCtx(sctx base.PlanContext, isNonPrep bool, markers []ast.ParamMarkerExpr, params []expression.Expression) error { + vars := sctx.GetSessionVars() + vars.PlanCacheParams.Reset() + for i, usingParam := range params { + val, err := usingParam.Eval(sctx.GetExprCtx().GetEvalCtx(), chunk.Row{}) + if err != nil { + return err + } + if isGetVarBinaryLiteral(sctx, usingParam) { + binVal, convErr := val.ToBytes() + if convErr != nil { + return convErr + } + val.SetBinaryLiteral(binVal) + } + if markers != nil { + param := markers[i].(*driver.ParamMarkerExpr) + param.Datum = val + param.InExecute = true + } + vars.PlanCacheParams.Append(val) + } + if vars.StmtCtx.EnableOptimizerDebugTrace && len(vars.PlanCacheParams.AllParamValues()) > 0 { + vals := vars.PlanCacheParams.AllParamValues() + valStrs := make([]string, len(vals)) + for i, val := range vals { + valStrs[i] = val.String() + } + debugtrace.RecordAnyValuesWithNames(sctx, "Parameter datums for EXECUTE", valStrs) + } + vars.PlanCacheParams.SetForNonPrepCache(isNonPrep) + return nil +} + +func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) error { + vars := sctx.GetSessionVars() + stmtAst := stmt.PreparedAst + vars.StmtCtx.StmtType = stmtAst.StmtType + + // step 1: check parameter number + if len(stmt.Params) != len(params) { + return errors.Trace(plannererrors.ErrWrongParamCount) + } + + // step 2: set parameter values + if err := SetParameterValuesIntoSCtx(sctx.GetPlanCtx(), isNonPrepared, stmt.Params, params); err != nil { + return errors.Trace(err) + } + + // step 3: add metadata lock and check each table's schema version + schemaNotMatch := false + for i := 0; i < len(stmt.dbName); i++ { + tbl, ok := is.TableByID(stmt.tbls[i].Meta().ID) + if !ok { + tblByName, err := is.TableByName(context.Background(), stmt.dbName[i], stmt.tbls[i].Meta().Name) + if err != nil { + return plannererrors.ErrSchemaChanged.GenWithStack("Schema change caused error: %s", err.Error()) + } + delete(stmt.RelateVersion, stmt.tbls[i].Meta().ID) + stmt.tbls[i] = tblByName + stmt.RelateVersion[tblByName.Meta().ID] = tblByName.Meta().Revision + } + newTbl, err := tryLockMDLAndUpdateSchemaIfNecessary(ctx, sctx.GetPlanCtx(), stmt.dbName[i], stmt.tbls[i], is) + if err != nil { + schemaNotMatch = true + continue + } + // The revision of tbl and newTbl may not be the same. + // Example: + // The version of stmt.tbls[i] is taken from the prepare statement and is revision v1. + // When stmt.tbls[i] is locked in MDL, the revision of newTbl is also v1. + // The revision of tbl is v2. The reason may have other statements trigger "tryLockMDLAndUpdateSchemaIfNecessary" before, leading to tbl revision update. + if stmt.tbls[i].Meta().Revision != newTbl.Meta().Revision || (tbl != nil && tbl.Meta().Revision != newTbl.Meta().Revision) { + schemaNotMatch = true + } + stmt.tbls[i] = newTbl + stmt.RelateVersion[newTbl.Meta().ID] = newTbl.Meta().Revision + } + + // step 4: check schema version + if schemaNotMatch || stmt.SchemaVersion != is.SchemaMetaVersion() { + // In order to avoid some correctness issues, we have to clear the + // cached plan once the schema version is changed. + // Cached plan in prepared struct does NOT have a "cache key" with + // schema version like prepared plan cache key + stmt.PointGet.pointPlan = nil + stmt.PointGet.planCacheKey = "" + stmt.PointGet.columnNames = nil + stmt.PointGet.pointPlanHints = nil + stmt.PointGet.Executor = nil + stmt.PointGet.ColumnInfos = nil + // If the schema version has changed we need to preprocess it again, + // if this time it failed, the real reason for the error is schema changed. + // Example: + // When running update in prepared statement's schema version distinguished from the one of execute statement + // We should reset the tableRefs in the prepared update statements, otherwise, the ast nodes still hold the old + // tableRefs columnInfo which will cause chaos in logic of trying point get plan. (should ban non-public column) + ret := &PreprocessorReturn{InfoSchema: is} + err := Preprocess(ctx, sctx, stmtAst.Stmt, InPrepare, WithPreprocessorReturn(ret)) + if err != nil { + return plannererrors.ErrSchemaChanged.GenWithStack("Schema change caused error: %s", err.Error()) + } + stmt.SchemaVersion = is.SchemaMetaVersion() + } + + // step 5: handle expiration + // If the lastUpdateTime less than expiredTimeStamp4PC, + // it means other sessions have executed 'admin flush instance plan_cache'. + // So we need to clear the current session's plan cache. + // And update lastUpdateTime to the newest one. + expiredTimeStamp4PC := domain.GetDomain(sctx).ExpiredTimeStamp4PC() + if stmt.StmtCacheable && expiredTimeStamp4PC.Compare(vars.LastUpdateTime4PC) > 0 { + sctx.GetSessionPlanCache().DeleteAll() + vars.LastUpdateTime4PC = expiredTimeStamp4PC + } + + // step 6: initialize the tableInfo2UnionScan, which indicates which tables are dirty. + for _, tbl := range stmt.tbls { + tblInfo := tbl.Meta() + if tableHasDirtyContent(sctx.GetPlanCtx(), tblInfo) { + sctx.GetSessionVars().StmtCtx.TblInfo2UnionScan[tblInfo] = true + } + } + + return nil +} + +// GetPlanFromPlanCache is the entry point of Plan Cache. +// It tries to get a valid cached plan from plan cache. +// If there is no such a plan, it'll call the optimizer to generate a new one. +// isNonPrepared indicates whether to use the non-prepared plan cache or the prepared plan cache. +func GetPlanFromPlanCache(ctx context.Context, sctx sessionctx.Context, + isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, + params []expression.Expression) (plan base.Plan, names []*types.FieldName, err error) { + if err := planCachePreprocess(ctx, sctx, isNonPrepared, is, stmt, params); err != nil { + return nil, nil, err + } + + sessVars := sctx.GetSessionVars() + stmtCtx := sessVars.StmtCtx + cacheEnabled := false + if isNonPrepared { + stmtCtx.SetCacheType(contextutil.SessionNonPrepared) + cacheEnabled = sessVars.EnableNonPreparedPlanCache // plan-cache might be disabled after prepare. + } else { + stmtCtx.SetCacheType(contextutil.SessionPrepared) + cacheEnabled = sessVars.EnablePreparedPlanCache + } + if stmt.StmtCacheable && cacheEnabled { + stmtCtx.EnablePlanCache() + } + if stmt.UncacheableReason != "" { + stmtCtx.WarnSkipPlanCache(stmt.UncacheableReason) + } + + var cacheKey, binding, reason string + var cacheable bool + if stmtCtx.UseCache() { + cacheKey, binding, cacheable, reason, err = NewPlanCacheKey(sctx, stmt) + if err != nil { + return nil, nil, err + } + if !cacheable { + stmtCtx.SetSkipPlanCache(reason) + } + } + + var paramTypes []*types.FieldType + if stmtCtx.UseCache() { + var cachedVal *PlanCacheValue + var hit, isPointPlan bool + if stmt.PointGet.pointPlan != nil && stmt.PointGet.planCacheKey == cacheKey { // if it's PointGet Plan, no need to use paramTypes + cachedVal = &PlanCacheValue{ + Plan: stmt.PointGet.pointPlan, + OutputColumns: stmt.PointGet.columnNames, + stmtHints: stmt.PointGet.pointPlanHints, + } + isPointPlan, hit = true, true + } else { + paramTypes = parseParamTypes(sctx, params) + cachedVal, hit = lookupPlanCache(ctx, sctx, cacheKey, paramTypes) + } + if hit { + if plan, names, ok, err := adjustCachedPlan(ctx, sctx, cachedVal, isNonPrepared, isPointPlan, binding, is, stmt); err != nil || ok { + return plan, names, err + } + } + } + if paramTypes == nil { + paramTypes = parseParamTypes(sctx, params) + } + + return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, cacheKey, paramTypes) +} + +func lookupPlanCache(ctx context.Context, sctx sessionctx.Context, cacheKey string, paramTypes []*types.FieldType) (cachedVal *PlanCacheValue, hit bool) { + if sctx.GetSessionVars().EnableInstancePlanCache { + if v, hit := domain.GetDomain(sctx).GetInstancePlanCache().Get(cacheKey, paramTypes); hit { + cachedVal = v.(*PlanCacheValue) + return cachedVal.CloneForInstancePlanCache(ctx, sctx.GetPlanCtx()) // clone the value to solve concurrency problem + } + } else { + if v, hit := sctx.GetSessionPlanCache().Get(cacheKey, paramTypes); hit { + return v.(*PlanCacheValue), true + } + } + return nil, false +} + +func adjustCachedPlan(ctx context.Context, sctx sessionctx.Context, cachedVal *PlanCacheValue, isNonPrepared, isPointPlan bool, + bindSQL string, is infoschema.InfoSchema, stmt *PlanCacheStmt) (base.Plan, + []*types.FieldName, bool, error) { + sessVars := sctx.GetSessionVars() + stmtCtx := sessVars.StmtCtx + if !isPointPlan { // keep the prior behavior + if err := checkPreparedPriv(ctx, sctx, stmt, is); err != nil { + return nil, nil, false, err + } + } + if !RebuildPlan4CachedPlan(cachedVal.Plan) { + return nil, nil, false, nil + } + sessVars.FoundInPlanCache = true + if len(bindSQL) > 0 { // We're using binding, set this to true. + sessVars.FoundInBinding = true + } + if metrics.ResettablePlanCacheCounterFortTest { + metrics.PlanCacheCounter.WithLabelValues("prepare").Inc() + } else { + core_metrics.GetPlanCacheHitCounter(isNonPrepared).Inc() + } + stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest) + stmtCtx.StmtHints = *cachedVal.stmtHints + return cachedVal.Plan, cachedVal.OutputColumns, true, nil +} + +// generateNewPlan call the optimizer to generate a new plan for current statement +// and try to add it to cache +func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema, + stmt *PlanCacheStmt, cacheKey string, paramTypes []*types.FieldType) (base.Plan, []*types.FieldName, error) { + stmtAst := stmt.PreparedAst + sessVars := sctx.GetSessionVars() + stmtCtx := sessVars.StmtCtx + + core_metrics.GetPlanCacheMissCounter(isNonPrepared).Inc() + sctx.GetSessionVars().StmtCtx.InPreparedPlanBuilding = true + p, names, err := OptimizeAstNode(ctx, sctx, stmtAst.Stmt, is) + sctx.GetSessionVars().StmtCtx.InPreparedPlanBuilding = false + if err != nil { + return nil, nil, err + } + + // check whether this plan is cacheable. + if stmtCtx.UseCache() { + if cacheable, reason := isPlanCacheable(sctx.GetPlanCtx(), p, len(paramTypes), len(stmt.limits), stmt.hasSubquery); !cacheable { + stmtCtx.SetSkipPlanCache(reason) + } + } + + // put this plan into the plan cache. + if stmtCtx.UseCache() { + cached := NewPlanCacheValue(p, names, paramTypes, &stmtCtx.StmtHints) + stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p) + stmtCtx.SetPlan(p) + stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest) + if sessVars.EnableInstancePlanCache { + domain.GetDomain(sctx).GetInstancePlanCache().Put(cacheKey, cached, paramTypes) + } else { + sctx.GetSessionPlanCache().Put(cacheKey, cached, paramTypes) + } + if _, ok := p.(*PointGetPlan); ok { + stmt.PointGet.pointPlan = p + stmt.PointGet.columnNames = names + stmt.PointGet.pointPlanHints = stmtCtx.StmtHints.Clone() + stmt.PointGet.planCacheKey = cacheKey + } + } + sessVars.FoundInPlanCache = false + return p, names, err +} + +// checkPreparedPriv checks the privilege of the prepared statement +func checkPreparedPriv(ctx context.Context, sctx sessionctx.Context, stmt *PlanCacheStmt, is infoschema.InfoSchema) error { + if pm := privilege.GetPrivilegeManager(sctx); pm != nil { + visitInfo := VisitInfo4PrivCheck(ctx, is, stmt.PreparedAst.Stmt, stmt.VisitInfos) + if err := CheckPrivilege(sctx.GetSessionVars().ActiveRoles, pm, visitInfo); err != nil { + return err + } + } + err := CheckTableLock(sctx, is, stmt.VisitInfos) + return err +} + +// IsSafeToReusePointGetExecutor checks whether this is a PointGet Plan and safe to reuse its executor. +func IsSafeToReusePointGetExecutor(sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanCacheStmt) bool { + if staleread.IsStmtStaleness(sctx) { + return false + } + // check auto commit + if !IsAutoCommitTxn(sctx.GetSessionVars()) { + return false + } + if stmt.SchemaVersion != is.SchemaMetaVersion() { + return false + } + return true +} diff --git a/pkg/planner/core/plan_cache_test.go b/pkg/planner/core/plan_cache_test.go new file mode 100644 index 0000000000000..f29c10bfe369c --- /dev/null +++ b/pkg/planner/core/plan_cache_test.go @@ -0,0 +1,1930 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core_test + +import ( + "context" + "errors" + "fmt" + "math/rand" + "strings" + "sync" + "testing" + "time" + + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner" + plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/testkit/testdata" + "github.com/pingcap/tidb/pkg/types" + driver "github.com/pingcap/tidb/pkg/types/parser_driver" + "github.com/pingcap/tidb/pkg/util" + "github.com/stretchr/testify/require" +) + +func TestInitLRUWithSystemVar(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@session.tidb_prepared_plan_cache_size = 0") // MinValue: 1 + tk.MustQuery("select @@session.tidb_prepared_plan_cache_size").Check(testkit.Rows("1")) + sessionVar := tk.Session().GetSessionVars() + + lru := plannercore.NewLRUPlanCache(uint(sessionVar.PreparedPlanCacheSize), 0, 0, tk.Session(), false) + require.NotNil(t, lru) +} + +func TestNonPreparedPlanCachePlanString(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int, b int, key(a))`) + tk.MustExec(`set @@tidb_enable_non_prepared_plan_cache=1`) + + ctx := tk.Session() + planString := func(sql string) string { + stmts, err := session.Parse(ctx, sql) + require.NoError(t, err) + stmt := stmts[0] + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(context.Background(), ctx, stmt, plannercore.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) + require.NoError(t, err) + return plannercore.ToString(p) + } + + require.Equal(t, planString("select a from t where a < 1"), "IndexReader(Index(t.a)[[-inf,1)])") + require.Equal(t, planString("select a from t where a < 10"), "IndexReader(Index(t.a)[[-inf,10)])") // range 1 -> 10 + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + require.Equal(t, planString("select * from t where b < 1"), "TableReader(Table(t)->Sel([lt(test.t.b, 1)]))") + require.Equal(t, planString("select * from t where b < 10"), "TableReader(Table(t)->Sel([lt(test.t.b, 10)]))") // filter 1 -> 10 + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) +} + +func TestNonPreparedPlanCacheInformationSchema(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_non_prepared_plan_cache=1") + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable(), plannercore.MockUnsignedTable()}) + + stmt, err := p.ParseOneStmt("select avg(a),avg(b),avg(c) from t", "", "") + require.NoError(t, err) + err = plannercore.Preprocess(context.Background(), tk.Session(), stmt, plannercore.WithPreprocessorReturn(&plannercore.PreprocessorReturn{InfoSchema: is})) + require.NoError(t, err) // no error + _, _, err = planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) // no error + _, _, err = planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) // no error + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) +} + +func TestNonPreparedPlanTypeRandomly(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t1 (a int, b int, key(a))`) + tk.MustExec(`create table t2 (a varchar(8), b varchar(8), key(a))`) + tk.MustExec(`create table t3 (a double, b double, key(a))`) + tk.MustExec(`create table t4 (a decimal(4, 2), b decimal(4, 2), key(a))`) + tk.MustExec(`create table t5 (a year, b year, key(a))`) + tk.MustExec(`create table t6 (a date, b date, key(a))`) + tk.MustExec(`create table t7 (a datetime, b datetime, key(a))`) + + n := 30 + for i := 0; i < n; i++ { + tk.MustExec(fmt.Sprintf(`insert into t1 values (%v, %v)`, randNonPrepTypeVal(t, n, "int"), randNonPrepTypeVal(t, n, "int"))) + tk.MustExec(fmt.Sprintf(`insert into t2 values (%v, %v)`, randNonPrepTypeVal(t, n, "varchar"), randNonPrepTypeVal(t, n, "varchar"))) + tk.MustExec(fmt.Sprintf(`insert into t3 values (%v, %v)`, randNonPrepTypeVal(t, n, "double"), randNonPrepTypeVal(t, n, "double"))) + tk.MustExec(fmt.Sprintf(`insert into t4 values (%v, %v)`, randNonPrepTypeVal(t, n, "decimal"), randNonPrepTypeVal(t, n, "decimal"))) + // TODO: fix it later + //tk.MustExec(fmt.Sprintf(`insert into t5 values (%v, %v)`, randNonPrepTypeVal(t, n, "year"), randNonPrepTypeVal(t, n, "year"))) + tk.MustExec(fmt.Sprintf(`insert into t6 values (%v, %v)`, randNonPrepTypeVal(t, n, "date"), randNonPrepTypeVal(t, n, "date"))) + tk.MustExec(fmt.Sprintf(`insert into t7 values (%v, %v)`, randNonPrepTypeVal(t, n, "datetime"), randNonPrepTypeVal(t, n, "datetime"))) + } + + for i := 0; i < 200; i++ { + q := fmt.Sprintf(`select * from t%v where %v`, rand.Intn(7)+1, randNonPrepFilter(t, n)) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + r0 := tk.MustQuery(q).Sort() // the first execution + tk.MustQuery(q).Sort().Check(r0.Rows()) // may hit the cache + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`) + tk.MustQuery(q).Sort().Check(r0.Rows()) // disable the non-prep cache + } +} + +func randNonPrepFilter(t *testing.T, scale int) string { + switch rand.Intn(4) { + case 0: // >= + return fmt.Sprintf(`a >= %v`, randNonPrepVal(t, scale)) + case 1: // < + return fmt.Sprintf(`a < %v`, randNonPrepVal(t, scale)) + case 2: // = + return fmt.Sprintf(`a = %v`, randNonPrepVal(t, scale)) + case 3: // in + return fmt.Sprintf(`a in (%v, %v)`, randNonPrepVal(t, scale), randNonPrepVal(t, scale)) + } + require.Error(t, errors.New("")) + return "" +} + +func randNonPrepVal(t *testing.T, scale int) string { + return randNonPrepTypeVal(t, scale, [7]string{"int", "varchar", "double", + "decimal", "year", "datetime", "date"}[rand.Intn(7)]) +} + +func randNonPrepTypeVal(t *testing.T, scale int, typ string) string { + switch typ { + case "int": + return fmt.Sprintf("%v", rand.Intn(scale)-(scale/2)) + case "varchar": + return fmt.Sprintf("'%v'", rand.Intn(scale)-(scale/2)) + case "double", "decimal": + return fmt.Sprintf("%v", float64(rand.Intn(scale)-(scale/2))/float64(10)) + case "year": + return fmt.Sprintf("%v", 2000+rand.Intn(scale)) + case "date": + return fmt.Sprintf("'2023-01-%02d'", rand.Intn(scale)+1) + case "timestamp", "datetime": + return fmt.Sprintf("'2023-01-01 00:00:%02d'", rand.Intn(scale)) + default: + require.Error(t, errors.New(typ)) + return "" + } +} + +func TestNonPreparedPlanCacheBasically(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int, b int, c int, d int, key(b), key(c, d))`) + for i := 0; i < 20; i++ { + tk.MustExec(fmt.Sprintf("insert into t values (%v, %v, %v, %v)", i, rand.Intn(20), rand.Intn(20), rand.Intn(20))) + } + + queries := []string{ + "select * from t where a<10", + "select * from t where a<13 and b<15", + "select * from t where b=13", + "select * from t where c<8", + "select * from t where d>8", + "select * from t where c=8 and d>10", + "select * from t where a<12 and b<13 and c<12 and d>2", + "select * from t where a in (1, 2, 3)", + "select * from t where a<13 or b<15", + "select * from t where a<13 or b<15 and c=13", + "select * from t where a in (1, 2)", + "select * from t where a in (1, 2) and b in (1, 2, 3)", + "select * from t where a in (1, 2) and b < 15", + "select * from t where a between 1 and 10", + "select * from t where a between 1 and 10 and b < 15", + } + + for _, query := range queries { + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`) + resultNormal := tk.MustQuery(query).Sort() + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + tk.MustQuery(query) // first process + tk.MustQuery(query).Sort().Check(resultNormal.Rows()) // equal to the result without plan-cache + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) // this plan is from plan-cache + } +} + +func TestNonPreparedPlanCacheInternalSQL(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec("create table t(a int, index(a))") + tk.MustExec("set tidb_enable_non_prepared_plan_cache=1") + + tk.MustExec("select * from t where a=1") + tk.MustExec("select * from t where a=1") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnOthers) + tk.Session().GetSessionVars().InRestrictedSQL = true + tk.MustExecWithContext(ctx, "select * from t where a=1") + tk.MustQueryWithContext(ctx, "select @@last_plan_from_cache").Check(testkit.Rows("0")) + + tk.Session().GetSessionVars().InRestrictedSQL = false + tk.MustExec("select * from t where a=1") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) +} + +func TestIssue53872(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec(`use test`) + tk.MustExec(`create table test(id int, col int)`) + tk.MustExec(`prepare stmt from "select id, ? as col1 from test where col=? group by id,col1"`) + tk.MustExec(`set @a=100, @b=100`) + tk.MustQuery(`execute stmt using @a,@b`).Check(testkit.Rows()) // no error + tk.MustQuery(`execute stmt using @a,@b`).Check(testkit.Rows()) +} + +func TestIssue38269(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set @@tidb_enable_prepared_plan_cache=1`) + tk.MustExec("set @@tidb_enable_collect_execution_info=0") + tk.MustExec(`set @@tidb_opt_advanced_join_hint=0`) + tk.MustExec("use test") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2(a int, b int, c int, index idx(a, b))") + tk.MustExec("prepare stmt1 from 'select /*+ inl_join(t2) */ * from t1 join t2 on t1.a = t2.a where t2.b in (?, ?, ?)'") + tk.MustExec("set @a = 10, @b = 20, @c = 30, @d = 40, @e = 50, @f = 60") + tk.MustExec("execute stmt1 using @a, @b, @c") + tk.MustExec("execute stmt1 using @d, @e, @f") + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Contains(t, rows[6][4], "range: decided by [eq(test.t2.a, test.t1.a) in(test.t2.b, 40, 50, 60)]") +} + +func TestIssue38533(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, key (a))") + tk.MustExec(`prepare st from "select /*+ use_index(t, a) */ a from t where a=? and a=?"`) + tk.MustExec(`set @a=1`) + tk.MustExec(`execute st using @a, @a`) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + plan := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.True(t, strings.Contains(plan[1][0].(string), "RangeScan")) // range-scan instead of full-scan + + tk.MustExec(`execute st using @a, @a`) + tk.MustExec(`execute st using @a, @a`) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) +} + +func TestInvalidRange(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, key(a))") + tk.MustExec("prepare st from 'select * from t where a>? and a (select max(a) from t t2 where t2.b < t1.b and t2.b < ?)", []int{1}, "0", false}, // scala + {"select * from t t1 where t1.a > (select 1 from t t2 where t2.b", "<", ">=", "<=", "in", "is null"} + op := ops[rand.Intn(len(ops))] + if op == "in" { + return fmt.Sprintf("%v %v (%v, %v, %v)", col, op, v(), v(), v()) + } else if op == "is null" { + return fmt.Sprintf("%v %v", col, op) + } + return fmt.Sprintf("%v %v %v", col, op, v()) + } + var queries []string + for i := 0; i < 50; i++ { + queries = append(queries, fmt.Sprintf("select * from t1 where %v", f())) + queries = append(queries, fmt.Sprintf("select * from t1 where %v and %v", f(), f())) + queries = append(queries, fmt.Sprintf("select * from t1 where %v and %v and %v", f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t1 where %v and %v and %v and %v", f(), f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t1 where %v and %v or %v", f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t1 where %v and %v or %v and %v", f(), f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v", f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v and %v", f(), f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v and %v and %v", f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v and %v and %v and %v", f(), f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v and %v or %v", f(), f(), f())) + queries = append(queries, fmt.Sprintf("select * from t2 where %v and %v or %v and %v", f(), f(), f(), f())) + } + return queries +} + +func planCacheIntConvertQueries(isNonPrep bool) []string { + cols := []string{"a", "b", "c", "d"} + ops := []string{"=", ">", "<", ">=", "<=", "in", "is null"} + v := func() string { + var val string + switch rand.Intn(3) { + case 0: + val = fmt.Sprintf("%v", 2000+rand.Intn(20)-10) + case 1: + val = fmt.Sprintf("%v.0", 2000+rand.Intn(20)-10) + default: + val = fmt.Sprintf("'%v'", 2000+rand.Intn(20)-10) + } + if !isNonPrep { + val = fmt.Sprintf("#?%v#", val) + } + return val + } + f := func() string { + col := cols[rand.Intn(len(cols))] + op := ops[rand.Intn(len(ops))] + if op == "is null" { + return fmt.Sprintf("%v is null", col) + } else if op == "in" { + if rand.Intn(2) == 0 { + return fmt.Sprintf("%v in (%v)", col, v()) + } + return fmt.Sprintf("%v in (%v, %v, %v)", col, v(), v(), v()) + } + return fmt.Sprintf("%v %v %v", col, op, v()) + } + fields := func() string { + var fs []string + for _, f := range []string{"a", "b", "c", "d"} { + if rand.Intn(4) == 0 { + continue + } + fs = append(fs, f) + } + if len(fs) == 0 { + return "*" + } + return strings.Join(fs, ", ") + } + var queries []string + for i := 0; i < 50; i++ { + queries = append(queries, fmt.Sprintf("select %v from t where %v", fields(), f())) + queries = append(queries, fmt.Sprintf("select %v from t where %v and %v", fields(), f(), f())) + queries = append(queries, fmt.Sprintf("select %v from t where %v and %v and %v", fields(), f(), f(), f())) + queries = append(queries, fmt.Sprintf("select %v from t where %v or %v", fields(), f(), f())) + queries = append(queries, fmt.Sprintf("select %v from t where %v or %v or %v", fields(), f(), f(), f())) + queries = append(queries, fmt.Sprintf("select %v from t where %v and %v or %v", fields(), f(), f(), f())) + } + return queries +} + +func planCacheIntConvertPrepareData(tk *testkit.TestKit) { + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists t`) + tk.MustExec(`create table t(a int, b year, c double, d varchar(16), key(a), key(b), key(c))`) + vals := make([]string, 0, 50) + for i := 0; i < 50; i++ { + a := fmt.Sprintf("%v", 2000+rand.Intn(20)-10) + if rand.Intn(10) == 0 { + a = "null" + } + b := fmt.Sprintf("%v", 2000+rand.Intn(20)-10) + if rand.Intn(10) == 0 { + b = "null" + } + c := fmt.Sprintf("%v.0", 2000+rand.Intn(20)-10) + if rand.Intn(10) == 0 { + c = "null" + } + d := fmt.Sprintf("'%v'", 2000+rand.Intn(20)-10) + if rand.Intn(10) == 0 { + d = "null" + } + vals = append(vals, fmt.Sprintf("(%s, %s, %s, %s)", a, b, c, d)) + } + tk.MustExec("insert into t values " + strings.Join(vals, ",")) +} + +func planCacheIndexMergeQueries(isNonPrep bool) []string { + ops := []string{"=", ">", "<", ">=", "<=", "in", "mod", "is null"} + f := func(col string) string { + n := rand.Intn(20) - 10 + nStr := fmt.Sprintf("%v", n) + if !isNonPrep { + nStr = fmt.Sprintf("#?%v#", n) + } + + op := ops[rand.Intn(len(ops))] + if op == "in" { + switch rand.Intn(3) { + case 0: // 1 element + return fmt.Sprintf("%s %s (%s)", col, op, nStr) + case 1: // multiple same elements + return fmt.Sprintf("%s %s (%s, %s, %s)", col, op, nStr, nStr, nStr) + default: // multiple different elements + if isNonPrep { + return fmt.Sprintf("%s %s (%d, %d)", col, op, n, n+1) + } + return fmt.Sprintf("%s %s (#?%d#, #?%d#)", col, op, n, n+1) + } + } else if op == "mod" { // this filter cannot be used to build range + return fmt.Sprintf("mod(%s, %s)=0", col, nStr) + } else if op == "is null" { + return fmt.Sprintf("%s %s", col, op) + } else { + return fmt.Sprintf("%s %s %s", col, op, nStr) + } + } + fields := func() string { + switch rand.Intn(5) { + case 0: + return "a" + case 1: + return "a, b" + case 2: + return "a, c" + case 3: + return "d" + default: + return "*" + } + } + var queries []string + for i := 0; i < 50; i++ { + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b) */ %s from t where %s and %s", fields(), f("a"), f("b"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, c) */ %s from t where %s and %s", fields(), f("a"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b, c) */ %s from t where %s and %s and %s", fields(), f("a"), f("b"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b) */ %s from t where %s or %s", fields(), f("a"), f("b"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, c) */ %s from t where %s or %s", fields(), f("a"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b, c) */ %s from t where %s or %s or %s", fields(), f("a"), f("b"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b) */ %s from t where %s and %s and %s", fields(), f("a"), f("a"), f("b"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, c) */ %s from t where %s and %s and %s", fields(), f("a"), f("c"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b, c) */ %s from t where %s and %s and %s and %s", fields(), f("a"), f("b"), f("b"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b) */ %s from t where (%s and %s) or %s", fields(), f("a"), f("a"), f("b"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, c) */ %s from t where %s or (%s and %s)", fields(), f("a"), f("c"), f("c"))) + queries = append(queries, fmt.Sprintf("select /*+ use_index_merge(t, a, b, c) */ %s from t where %s or (%s and %s) or %s", fields(), f("a"), f("b"), f("b"), f("c"))) + } + return queries +} + +func planCacheIndexMergePrepareData(tk *testkit.TestKit) { + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, d int, key(a), key(b), key(c))") + vals := make([]string, 0, 50) + v := func() string { + if rand.Intn(10) == 0 { + return "null" + } + return fmt.Sprintf("%d", rand.Intn(20)-10) + } + for i := 0; i < 50; i++ { + vals = append(vals, fmt.Sprintf("(%s, %s, %s, %s)", v(), v(), v(), v())) + } + tk.MustExec("insert into t values " + strings.Join(vals, ",")) +} + +func TestPlanCacheRandomCases(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + testRandomPlanCacheCases(t, planCacheIndexMergePrepareData, planCacheIndexMergeQueries) + testRandomPlanCacheCases(t, planCacheIntConvertPrepareData, planCacheIntConvertQueries) + testRandomPlanCacheCases(t, planCachePointGetPrepareData, planCachePointGetQueries) +} + +func testRandomPlanCacheCases(t *testing.T, + prepFunc func(tk *testkit.TestKit), + queryFunc func(isNonPrep bool) []string) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + prepFunc(tk) + + // prepared plan cache + for _, q := range queryFunc(true) { + tk.MustExec("set tidb_enable_non_prepared_plan_cache=0") + result1 := tk.MustQuery(q).Sort() + tk.MustExec("set tidb_enable_non_prepared_plan_cache=1") + result2 := tk.MustQuery(q).Sort() + require.True(t, result1.Equal(result2.Rows())) + } + + // non prepared plan cache + for _, q := range queryFunc(false) { + q, prepStmt, parameters := convertQueryToPrepExecStmt(q) + result1 := tk.MustQuery(q).Sort() + tk.MustExec(prepStmt) + var xs []string + for i, p := range parameters { + tk.MustExec(fmt.Sprintf("set @x%d = %s", i, p)) + xs = append(xs, fmt.Sprintf("@x%d", i)) + } + var execStmt string + if len(xs) == 0 { + execStmt = "execute st" + } else { + execStmt = fmt.Sprintf("execute st using %s", strings.Join(xs, ", ")) + } + result2 := tk.MustQuery(execStmt).Sort() + require.True(t, result1.Equal(result2.Rows())) + } +} + +func TestPlanCacheSubquerySPMEffective(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + + testCases := []struct { + sql string + params []int + }{ + {"select * from t t1 where exists (select /*/ 1 from t t2 where t2.b < t1.b and t2.b < ?)", []int{1}}, // exist + {"select * from t t1 where exists (select /*/ b from t t2 where t1.a = t2.a and t2.b < ? limit ?)", []int{1, 1}}, + {"select * from t t1 where t1.a in (select /*/ a from t t2 where t2.a > ? and t1.a = t2.a)", []int{1}}, + {"select * from t t1 where t1.a < (select /*/ sum(t2.a) from t t2 where t2.b = t1.b and t2.a > ?)", []int{1}}, + } + + // hint + for _, testCase := range testCases { + sql := strings.Replace(testCase.sql, "/*/", "/*+ NO_DECORRELATE() */", 1) + tk.MustExec(fmt.Sprintf("prepare stmt from '%s'", sql)) + var using []string + for i, p := range testCase.params { + tk.MustExec(fmt.Sprintf("set @a%d = %d", i, p)) + using = append(using, fmt.Sprintf("@a%d", i)) + } + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } + tk.MustExec("deallocate prepare stmt") + + // binding before prepare + for _, testCase := range testCases { + sql := strings.Replace(testCase.sql, "/*/", "", 1) + bindSQL := strings.Replace(testCase.sql, "/*/", "/*+ NO_DECORRELATE() */", 1) + tk.MustExec("create binding for " + sql + " using " + bindSQL) + tk.MustExec(fmt.Sprintf("prepare stmt from '%s'", sql)) + var using []string + for i, p := range testCase.params { + tk.MustExec(fmt.Sprintf("set @a%d = %d", i, p)) + using = append(using, fmt.Sprintf("@a%d", i)) + } + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } + + // binding after prepare + for _, testCase := range testCases { + sql := strings.Replace(testCase.sql, "/*/", "", 1) + bindSQL := strings.Replace(testCase.sql, "/*/", "/*+ NO_DECORRELATE() */", 1) + tk.MustExec(fmt.Sprintf("prepare stmt from '%s'", sql)) + var using []string + for i, p := range testCase.params { + tk.MustExec(fmt.Sprintf("set @a%d = %d", i, p)) + using = append(using, fmt.Sprintf("@a%d", i)) + } + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustExec("create binding for " + sql + " using " + bindSQL) + tk.MustExec("execute stmt using " + strings.Join(using, ", ")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } +} + +func TestIssue42125(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, b int, c int, unique key(a, b))") + + // should use BatchPointGet + tk.MustExec("prepare st from 'select * from t where a=1 and b in (?, ?)'") + tk.MustExec("set @a=1, @b=2") + tk.MustExec("execute st using @a, @b") + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Equal(t, rows[0][0], "Batch_Point_Get_5") // use BatchPointGet + tk.MustExec("execute st using @a, @b") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: Batch/PointGet plans may be over-optimized")) + + // should use PointGet: unsafe PointGet + tk.MustExec("prepare st from 'select * from t where a=1 and b>=? and b<=?'") + tk.MustExec("set @a=1, @b=1") + tk.MustExec("execute st using @a, @b") + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Equal(t, rows[0][0], "Point_Get_5") // use Point_Get_5 + tk.MustExec("execute st using @a, @b") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // cannot hit + + // safe PointGet + tk.MustExec("prepare st from 'select * from t where a=1 and b=? and c Selection + require.Contains(t, rows[1][0], "Point_Get") + tk.MustExec("execute st using @a, @b") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) // can hit +} + +func TestNonPreparedPlanExplainWarning(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec(`create table t (a int, b int, c int, d int, e enum('1', '2', '3'), s set('1', '2', '3'), j json, bt bit(8), key(b), key(c, d))`) + tk.MustExec("create table t1(a int, b int, index idx_b(b)) 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, first_name varchar(50), last_name varchar(50), full_name varchar(101) generated always as (concat(first_name,' ',last_name)))") + tk.MustExec("create or replace SQL SECURITY INVOKER view v as select a from t") + tk.MustExec("analyze table t, t1, t2") // eliminate stats warnings + tk.MustExec("set @@session.tidb_enable_non_prepared_plan_cache = 1") + + supported := []string{ + "select * from t where a<10", + "select * from t where a<13 and b<15", + "select * from t where b=13", + "select * from t where c<8", + "select * from t where d>8", + "select * from t where c=8 and d>10", + "select * from t where a<12 and b<13 and c<12 and d>2", + "select * from t where a in (1, 2, 3)", + "select * from t where a<13 or b<15", + "select * from t where a<13 or b<15 and c=13", + "select * from t where a in (1, 2)", + "select * from t where a in (1, 2) and b in (1, 2, 3)", + "select * from t where a in (1, 2) and b < 15", + "select * from t where a between 1 and 10", + "select * from t where a between 1 and 10 and b < 15", + "select * from t where a+b=13", + "select * from t where mod(a, 3)=1", + "select * from t where d>now()", + "select distinct a from t1 where a > 1 and b < 2", // distinct + "select count(*) from t1 where a > 1 and b < 2 group by a", // group by + "select * from t1 order by a", // order by + "select * from t3 where full_name = 'a b'", // generated column + "select * from t3 where a > 1 and full_name = 'a b'", + "select * from t1 where a in (1, 2)", // Partitioned + "select * from t2 where a in (1, 2) and b in (1, 2, 3)", // Partitioned + "select * from t1 where a in (1, 2) and b < 15", // Partitioned + } + + unsupported := []string{ + "select /*+ use_index(t1, idx_b) */ * from t1 where a > 1 and b < 2", // hint + "select a, sum(b) as c from t1 where a > 1 and b < 2 group by a having sum(b) > 1", // having + "select * from (select * from t1) t", // sub-query + "select * from t1 where a in (select a from t)", // uncorrelated sub-query + "select * from t1 where a in (select a from t where a > t1.a)", // correlated sub-query + "select * from t where j < 1", // json + "select * from t where a > 1 and j < 1", + "select * from t where e < '1'", // enum + "select * from t where a > 1 and e < '1'", + "select * from t where s < '1'", // set + "select * from t where a > 1 and s < '1'", + "select * from t where bt > 0", // bit + "select * from t where a > 1 and bt > 0", + "select data_type from INFORMATION_SCHEMA.columns where table_name = 'v'", // memTable + "select * from v", // view + "select * from t where a = null", // null + "select * from t where false", // table dual + } + + reasons := []string{ + "skip non-prepared plan-cache: queries that have hints, having-clause, window-function are not supported", + "skip non-prepared plan-cache: queries that have hints, having-clause, window-function are not supported", + "skip non-prepared plan-cache: queries that have sub-queries are not supported", + "skip non-prepared plan-cache: query has some unsupported Node", + "skip non-prepared plan-cache: query has some unsupported Node", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: query has some filters with JSON, Enum, Set or Bit columns", + "skip non-prepared plan-cache: access tables in system schema", + "skip non-prepared plan-cache: queries that access views are not supported", + "skip non-prepared plan-cache: query has null constants", + "skip non-prepared plan-cache: some parameters may be overwritten when constant propagation", + } + + all := append(supported, unsupported...) + + explainFormats := []string{ + types.ExplainFormatBrief, + types.ExplainFormatDOT, + types.ExplainFormatHint, + types.ExplainFormatROW, + types.ExplainFormatVerbose, + types.ExplainFormatTraditional, + types.ExplainFormatBinary, + types.ExplainFormatTiDBJSON, + types.ExplainFormatCostTrace, + } + // all cases no warnings use other format + for _, q := range all { + tk.MustExec("explain " + q) + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("explain " + q) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } + for _, format := range explainFormats { + for _, q := range all { + tk.MustExec(fmt.Sprintf("explain format = '%v' %v", format, q)) + //tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustQuery("show warnings").CheckNotContain("plan cache") + tk.MustExec(fmt.Sprintf("explain format = '%v' %v", format, q)) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + } + } + + // unsupported case with warning use 'plan_cache' format + for idx, q := range unsupported { + tk.MustExec("explain format = 'plan_cache'" + q) + warn := tk.MustQuery("show warnings").Rows()[0] + require.Equal(t, reasons[idx], warn[2], "idx: %d", idx) + } +} + +func TestNonPreparedPlanCachePanic(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + + tk.MustExec("create table t (a varchar(255), b int, c char(10), primary key (c, a));") + ctx := tk.Session().(sessionctx.Context) + + s := parser.New() + for _, sql := range []string{ + "select 1 from t where a='x'", + "select * from t where c='x'", + "select * from t where a='x' and c='x'", + "select * from t where a='x' and c='x' and b=1", + } { + stmtNode, err := s.ParseOneStmt(sql, "", "") + require.NoError(t, err) + preprocessorReturn := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(context.Background(), ctx, stmtNode, plannercore.WithPreprocessorReturn(preprocessorReturn)) + require.NoError(t, err) + _, _, err = planner.Optimize(context.TODO(), ctx, stmtNode, preprocessorReturn.InfoSchema) + require.NoError(t, err) // not panic + } +} + +func TestNonPreparedPlanCacheAutoStmtRetry(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk1.MustExec("create table t(id int primary key, k int, UNIQUE KEY(k))") + tk1.MustExec("insert into t values(1, 1)") + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + tk2.MustExec("use test") + tk1.MustExec("begin") + tk1.MustExec("update t set k=3 where id=1") + + var wg sync.WaitGroup + var tk2Err error + wg.Add(1) + go func() { + // trigger statement auto-retry on tk2 + _, tk2Err = tk2.Exec("insert into t values(3, 3)") + wg.Done() + }() + time.Sleep(100 * time.Millisecond) + _, err := tk1.Exec("commit") + require.NoError(t, err) + wg.Wait() + require.ErrorContains(t, tk2Err, "Duplicate entry") +} + +func TestIssue43667Concurrency(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table cycle (pk int key, val int)") + var wg sync.WaitGroup + concurrency := 30 + for i := 0; i < concurrency; i++ { + tk.MustExec(fmt.Sprintf("insert into cycle values (%v,%v)", i, i)) + wg.Add(1) + go func(id int) { + defer wg.Done() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_non_prepared_plan_cache=1") + query := fmt.Sprintf("select (val) from cycle where pk = %v", id) + for j := 0; j < 5000; j++ { + tk.MustQuery(query).Check(testkit.Rows(fmt.Sprintf("%v", id))) + } + }(i) + } + wg.Wait() +} + +func TestIssue43667(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + tk.MustExec(`create table cycle (pk int not null primary key, sk int not null, val int)`) + tk.MustExec(`insert into cycle values (4, 4, 4)`) + tk.MustExec(`insert into cycle values (7, 7, 7)`) + + tk.MustQuery(`select (val) from cycle where pk = 4`).Check(testkit.Rows("4")) + tk.MustQuery(`select (val) from cycle where pk = 7`).Check(testkit.Rows("7")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + updateAST := func(stmt ast.StmtNode) { + v := stmt.(*ast.SelectStmt).Where.(*ast.BinaryOperationExpr).R.(*driver.ValueExpr) + v.Datum.SetInt64(7) + } + + tctx := context.WithValue(context.Background(), plannercore.PlanCacheKeyTestIssue43667{}, updateAST) + tk.MustQueryWithContext(tctx, `select (val) from cycle where pk = 4`).Check(testkit.Rows("4")) +} + +func TestIssue47133(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + tk.MustExec(`CREATE TABLE t (id int NOT NULL, personId int NOT NULL, + name varchar(255) NOT NULL, PRIMARY KEY (id, personId))`) + tk.MustExec(`insert into t values (1, 1, '')`) + + cnt := 0 + checkFieldNames := func(names []*types.FieldName) { + require.Equal(t, len(names), 2) + require.Equal(t, names[0].String(), "test.t.user_id") + require.Equal(t, names[1].String(), "test.t.user_personid") + cnt += 1 + } + tctx := context.WithValue(context.Background(), plannercore.PlanCacheKeyTestIssue47133{}, checkFieldNames) + tk.MustQueryWithContext(tctx, `SELECT id AS User_id, personId AS User_personId FROM t WHERE (id = 1 AND personId = 1)`).Check( + testkit.Rows("1 1")) + tk.MustQueryWithContext(tctx, `SELECT id AS User_id, personId AS User_personId FROM t WHERE (id = 1 AND personId = 1)`).Check( + testkit.Rows("1 1")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + require.Equal(t, cnt, 2) +} + +func TestPlanCacheBindingIgnore(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create database test1`) + tk.MustExec(`use test1`) + tk.MustExec(`create table t (a int)`) + tk.MustExec(`create database test2`) + tk.MustExec(`use test2`) + tk.MustExec(`create table t (a int)`) + + tk.MustExec(`prepare st1 from 'select * from test1.t'`) + tk.MustExec(`execute st1`) + tk.MustExec(`execute st1`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec(`prepare st2 from 'select * from test2.t'`) + tk.MustExec(`execute st2`) + tk.MustExec(`execute st2`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + tk.MustExec(`create global binding using select /*+ ignore_plan_cache() */ * from test1.t`) + tk.MustExec(`execute st1`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustExec(`execute st1`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustExec(`create global binding using select /*+ ignore_plan_cache() */ * from test2.t`) + tk.MustExec(`execute st2`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustExec(`execute st2`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) +} + +func TestIssue53505(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t (v varchar(16))`) + tk.MustExec(`insert into t values ('156')`) + tk.MustExec(`prepare stmt7 from 'select * from t where v = conv(?, 16, 8)'`) + tk.MustExec(`set @arg=0x6E`) + tk.MustQuery(`execute stmt7 using @arg`).Check(testkit.Rows("156")) + tk.MustQuery(`execute stmt7 using @arg`).Check(testkit.Rows("156")) + tk.MustExec(`set @arg=0x70`) + tk.MustQuery(`execute stmt7 using @arg`).Check(testkit.Rows()) // empty +} + +func TestBuiltinFuncFlen(t *testing.T) { + // same as TestIssue45378 and TestIssue45253 + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`CREATE TABLE t1(c1 INT)`) + tk.MustExec(`INSERT INTO t1 VALUES (1)`) + + funcs := []string{ast.Abs, ast.Acos, ast.Asin, ast.Atan, ast.Ceil, ast.Ceiling, ast.Cos, + ast.CRC32, ast.Degrees, ast.Floor, ast.Ln, ast.Log, ast.Log2, ast.Log10, ast.Unhex, + ast.Radians, ast.Rand, ast.Round, ast.Sign, ast.Sin, ast.Sqrt, ast.Tan, ast.SM3, + ast.Quote, ast.RTrim, ast.ToBase64, ast.Trim, ast.Upper, ast.Ucase, ast.Hex, + ast.BitLength, ast.CharLength, ast.Compress, ast.MD5, ast.SHA1, ast.SHA} + args := []string{"2038330881", "'2038330881'", "'牵'", "-1", "''", "0"} + + for _, f := range funcs { + for _, a := range args { + q := fmt.Sprintf("SELECT c1 from t1 where %s(%s)", f, a) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) + r1 := tk.MustQuery(q) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`) + r2 := tk.MustQuery(q) + r1.Sort().Check(r2.Sort().Rows()) + } + } +} + +func TestWarningWithDisablePlanCacheStmt(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int) partition by hash(a) partitions 4;") + tk.MustExec("analyze table t;") + tk.MustExec("prepare st from 'select * from t';") + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustExec("execute st;") + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustExec("execute st;") + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) +} + +func randValueForMVIndex(colType string) string { + randSize := 50 + colType = strings.ToLower(colType) + switch colType { + case "int": + return fmt.Sprintf("%v", randSize-rand.Intn(randSize)) + case "string": + return fmt.Sprintf("\"%v\"", rand.Intn(randSize)) + case "json-string": + var array []string + arraySize := 1 + rand.Intn(5) + for i := 0; i < arraySize; i++ { + array = append(array, randValueForMVIndex("string")) + } + return "'[" + strings.Join(array, ", ") + "]'" + case "json-signed": + var array []string + arraySize := 1 + rand.Intn(5) + for i := 0; i < arraySize; i++ { + array = append(array, randValueForMVIndex("int")) + } + return "'[" + strings.Join(array, ", ") + "]'" + default: + return "unknown type " + colType + } +} + +func insertValuesForMVIndex(nRows int, colTypes ...string) string { + var stmtVals []string + for i := 0; i < nRows; i++ { + var vals []string + for _, colType := range colTypes { + vals = append(vals, randValueForMVIndex(colType)) + } + stmtVals = append(stmtVals, "("+strings.Join(vals, ", ")+")") + } + return strings.Join(stmtVals, ", ") +} + +func verifyPlanCacheForMVIndex(t *testing.T, tk *testkit.TestKit, isIndexMerge, hitCache bool, queryTemplate string, colTypes ...string) { + for i := 0; i < 5; i++ { + var vals []string + for _, colType := range colTypes { + vals = append(vals, randValueForMVIndex(colType)) + } + + query := queryTemplate + var setStmt, usingStmt string + for i, p := range vals { + query = strings.Replace(query, "?", p, 1) + if i > 0 { + setStmt += ", " + usingStmt += ", " + } + setStmt += fmt.Sprintf("@a%v=%v", i, p) + usingStmt += fmt.Sprintf("@a%v", i) + } + result := tk.MustQuery(query).Sort() + if isIndexMerge { + tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning + } + tk.MustExec(fmt.Sprintf("set %v", setStmt)) + tk.MustExec(fmt.Sprintf("prepare stmt from '%v'", queryTemplate)) + if isIndexMerge { + tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning + } + result1 := tk.MustQuery(fmt.Sprintf("execute stmt using %v", usingStmt)).Sort() + result.Check(result1.Rows()) + if isIndexMerge && hitCache { + tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning + } + result2 := tk.MustQuery(fmt.Sprintf("execute stmt using %v", usingStmt)).Sort() + result.Check(result2.Rows()) + if isIndexMerge && hitCache { + tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning + } + result3 := tk.MustQuery(fmt.Sprintf("execute stmt using %v", usingStmt)).Sort() + result.Check(result3.Rows()) + if isIndexMerge && hitCache { + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) // hit the cache + } + + if isIndexMerge && hitCache { // check the plan + result4 := tk.MustQuery(fmt.Sprintf("execute stmt using %v", usingStmt)).Sort() + result.Check(result4.Rows()) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + haveIndexMerge := false + for _, r := range rows { + if strings.Contains(r[0].(string), "IndexMerge") { + haveIndexMerge = true + } + } + require.True(t, haveIndexMerge) // IndexMerge has to be used. + } + } +} + +func TestPlanCacheMVIndexRandomly(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`set @@tidb_opt_fix_control = "45798:on"`) + + // cases from TestIndexMergeFromComposedDNFCondition + tk.MustExec(`drop table if exists t2`) + tk.MustExec(`create table t2(a json, b json, c int, d int, e int, index idx(c, (cast(a as signed array))), index idx2((cast(b as signed array)), c), index idx3(c, d), index idx4(d))`) + tk.MustExec(fmt.Sprintf("insert into t2 values %v", insertValuesForMVIndex(100, "json-signed", "json-signed", "int", "int", "int"))) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where (? member of (a) and c=?) or (? member of (b) and c=?)`, + `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where (? member of (a) and c=? and d=?) or (? member of (b) and c=? and d=?)`, + `int`, `int`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, false, false, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where ( json_contains(a, ?) and c=? and d=?) or (? member of (b) and c=? and d=?)`, + `json-signed`, `int`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, false, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where ( json_overlaps(a, ?) and c=? and d=?) or (? member of (b) and c=? and d=?)`, + `json-signed`, `int`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx2, idx, idx4) */ * from t2 where ( json_contains(a, ?) and d=?) or (? member of (b) and c=? and d=?)`, + `json-signed`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where (? member of (a) and ? member of (b) and c=?) or (? member of (b) and c=?)`, + `int`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, false, true, + `select * from t2 where (? member of (a) and ? member of (b) and c=?) or (? member of (b) and c=?) or e=?`, + `int`, `int`, `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx2, idx, idx4) */ * from t2 where (? member of (a) and ? member of (b) and c=?) or (? member of (b) and c=?) or d=?`, + `int`, `int`, `int`, `int`, `int`, `int`) + + // cases from TestIndexMergeFromComposedCNFCondition + tk.MustExec(`drop table if exists t1, t2`) + tk.MustExec(`create table t1(a json, b json, c int, d int, index idx((cast(a as signed array))), index idx2((cast(b as signed array))))`) + tk.MustExec(fmt.Sprintf("insert into t1 values %v", insertValuesForMVIndex(100, "json-signed", "json-signed", "int", "int"))) + tk.MustExec(`create table t2(a json, b json, c int, d int, index idx(c, (cast(a as signed array))), index idx2((cast(b as signed array)), c), index idx3(c, d), index idx4(d))`) + tk.MustExec(fmt.Sprintf("insert into t2 values %v", insertValuesForMVIndex(100, "json-signed", "json-signed", "int", "int"))) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t1, idx, idx2) */ * from t1 where ? member of (a) and ? member of (b)`, + `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx, idx2) */ * from t2 where ? member of (a) and ? member of (b) and c=?`, + `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, true, + `select /*+ use_index_merge(t2, idx, idx2, idx4) */ * from t2 where ? member of (a) and ? member of (b) and c=? and d=?`, + `int`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, false, + `select /*+ use_index_merge(t2, idx2, idx, idx3) */ * from t2 where json_contains(a, ?) and c=? and ? member of (b) and d=?`, + `json-signed`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, true, false, + `select /*+ use_index_merge(t2, idx2, idx, idx3) */ * from t2 where json_overlaps(a, ?) and c=? and ? member of (b) and d=?`, + `json-signed`, `int`, `int`, `int`) + verifyPlanCacheForMVIndex(t, tk, false, true, + `select /*+ use_index_merge(t2, idx2, idx) */ * from t2 where ? member of (a) and c=? and c=?`, + `int`, `int`, `int`) + + // case from TestIndexMergeIssue50265 + tk.MustExec(`drop table if exists t`) + tk.MustExec("create table t(pk varbinary(255) NOT NULL, domains json null, image_signatures json null, canonical_links json null, fpi json null, KEY `domains` ((cast(`domains` as char(253) array))), KEY `image_signatures` ((cast(`image_signatures` as char(32) array))),KEY `canonical_links` ((cast(`canonical_links` as char(1000) array))), KEY `fpi` ((cast(`fpi` as signed array))))") + tk.MustExec(fmt.Sprintf("insert into t values %v", insertValuesForMVIndex(100, "string", "json-string", "json-string", "json-string", "json-signed"))) + verifyPlanCacheForMVIndex(t, tk, false, false, + `SELECT /*+ use_index_merge(t, domains, image_signatures, canonical_links, fpi) */ pk FROM t WHERE ? member of (domains) OR ? member of (image_signatures) OR ? member of (canonical_links) OR json_contains(fpi, "[69236881]") LIMIT 100`, + `string`, `string`, `string`) + + // case from TestIndexMergeEliminateRedundantAndPaths + tk.MustExec(`DROP table if exists t`) + tk.MustExec("CREATE TABLE `t` (`pk` varbinary(255) NOT NULL,`nslc` json DEFAULT NULL,`fpi` json DEFAULT NULL,`point_of_sale_country` varchar(2) DEFAULT NULL,KEY `fpi` ((cast(`fpi` as signed array))),KEY `nslc` ((cast(`nslc` as char(1000) array)),`point_of_sale_country`),KEY `nslc_old` ((cast(`nslc` as char(1000) array))))") + tk.MustExec(fmt.Sprintf("insert into t values %v", insertValuesForMVIndex(100, "string", "json-string", "json-signed", "string"))) + verifyPlanCacheForMVIndex(t, tk, true, true, + "SELECT /*+ use_index_merge(t, fpi, nslc_old, nslc) */ * FROM t WHERE ? member of (fpi) AND ? member of (nslc) LIMIT 100", + "int", "string") + + // case from TestIndexMergeSingleCaseCouldFeelIndexMergeHint + tk.MustExec(`DROP table if exists t`) + tk.MustExec("CREATE TABLE t (nslc json DEFAULT NULL,fpi json DEFAULT NULL,point_of_sale_country int,KEY nslc ((cast(nslc as char(1000) array)),point_of_sale_country),KEY fpi ((cast(fpi as signed array))))") + tk.MustExec(fmt.Sprintf("insert into t values %v", insertValuesForMVIndex(100, "json-string", "json-signed", "int"))) + verifyPlanCacheForMVIndex(t, tk, true, true, + "SELECT /*+ use_index_merge(t, nslc) */ * FROM t WHERE ? member of (fpi) AND ? member of (nslc) LIMIT 1", + "int", "string") + verifyPlanCacheForMVIndex(t, tk, true, true, + "SELECT /*+ use_index_merge(t, fpi) */ * FROM t WHERE ? member of (fpi) AND ? member of (nslc) LIMIT 1", + "int", "string") +} + +func TestPlanCacheMVIndexManually(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`set @@tidb_opt_fix_control = "45798:on"`) + + var ( + input []string + output []struct { + SQL string + Result []string + } + ) + planSuiteData := plannercore.GetPlanCacheSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i := range input { + testdata.OnRecord(func() { + output[i].SQL = input[i] + }) + if strings.HasPrefix(strings.ToLower(input[i]), "select") || + strings.HasPrefix(strings.ToLower(input[i]), "execute") || + strings.HasPrefix(strings.ToLower(input[i]), "show") { + result := tk.MustQuery(input[i]) + testdata.OnRecord(func() { + output[i].Result = testdata.ConvertRowsToStrings(result.Rows()) + }) + result.Check(testkit.Rows(output[i].Result...)) + } else { + tk.MustExec(input[i]) + } + } +} + +func BenchmarkPlanCacheBindingMatch(b *testing.B) { + store := testkit.CreateMockStore(b) + tk := testkit.NewTestKit(b, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, key(a))") + tk.MustExec(`create global binding using select * from t where a=1`) + + tk.MustExec(`prepare st from 'select * from t where a=?'`) + tk.MustExec(`set @a=1`) + b.ResetTimer() + for i := 0; i < b.N; i++ { + tk.MustExec("execute st using @a") + } +} + +func BenchmarkPlanCacheInsert(b *testing.B) { + store := testkit.CreateMockStore(b) + tk := testkit.NewTestKit(b, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int)") + + tk.MustExec("prepare st from 'insert into t values (1)'") + b.ResetTimer() + for i := 0; i < b.N; i++ { + tk.MustExec("execute st") + } +} + +func BenchmarkNonPreparedPlanCacheDML(b *testing.B) { + store := testkit.CreateMockStore(b) + tk := testkit.NewTestKit(b, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int)") + tk.MustExec("set tidb_enable_non_prepared_plan_cache=1") + + b.ResetTimer() + for i := 0; i < b.N; i++ { + tk.MustExec("insert into t values (1)") + tk.MustExec("update t set a = 2 where a = 1") + tk.MustExec("delete from t where a = 2") + } +} + +func TestPreparedPlanCachePartitions(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec(`create table t (a int primary key, b varchar(255)) partition by hash(a) partitions 3`) + tk.MustExec(`insert into t values (1,"a"),(2,"b"),(3,"c"),(4,"d"),(5,"e"),(6,"f")`) + tk.MustExec(`analyze table t`) + tk.MustExec(`prepare stmt from 'select a,b from t where a = ?;'`) + tk.MustExec(`set @a=1`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("1 a")) + // Same partition works, due to pruning is not affected + tk.MustExec(`set @a=4`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("4 d")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + // Different partition needs code changes + tk.MustExec(`set @a=2`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("2 b")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec(`prepare stmt2 from 'select b,a from t where a = ?;'`) + tk.MustExec(`set @a=1`) + tk.MustQuery(`execute stmt2 using @a`).Check(testkit.Rows("a 1")) + tk.MustExec(`set @a=3`) + tk.MustQuery(`execute stmt2 using @a`).Check(testkit.Rows("c 3")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec(`drop table t`) + + tk.MustExec(`create table t (a int primary key, b varchar(255), c varchar(255), key (b)) partition by range (a) (partition pNeg values less than (0), partition p0 values less than (1000000), partition p1M values less than (2000000))`) + tk.MustExec(`insert into t values (-10, -10, -10), (0, 0, 0), (-1, NULL, NULL), (1000, 1000, 1000), (1000000, 1000000, 1000000), (1500000, 1500000, 1500000), (1999999, 1999999, 1999999)`) + tk.MustExec(`analyze table t`) + tk.MustExec(`prepare stmt3 from 'select a,c,b from t where a = ?'`) + tk.MustExec(`set @a=2000000`) + // This should use TableDual + tk.MustQuery(`execute stmt3 using @a`).Check(testkit.Rows()) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).MultiCheckContain([]string{"Point_Get", "partition:dual", "handle:2000000"}) + tk.MustExec(`set @a=1999999`) + tk.MustQuery(`execute stmt3 using @a`).Check(testkit.Rows("1999999 1999999 1999999")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).MultiCheckContain([]string{"Point_Get", "partition:p1M", "handle:1999999"}) + tk.MustQuery(`execute stmt3 using @a`).Check(testkit.Rows("1999999 1999999 1999999")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + + tk.MustExec(`prepare stmt4 from 'select a,c,b from t where a IN (?,?,?)'`) + tk.MustExec(`set @a=1999999,@b=0,@c=-1`) + tk.MustQuery(`execute stmt4 using @a,@b,@c`).Sort().Check(testkit.Rows("-1 ", "0 0 0", "1999999 1999999 1999999")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`execute stmt4 using @a,@b,@c`).Sort().Check(testkit.Rows("-1 ", "0 0 0", "1999999 1999999 1999999")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) +} + +func TestPreparedPlanCachePartitionIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t (b varchar(255), a int primary key nonclustered, key (b)) partition by key(a) partitions 3`) + tk.MustExec(`insert into t values ('Ab', 1),('abc',2),('BC',3),('AC',4),('BA',5),('cda',6)`) + tk.MustExec(`analyze table t`) + tk.MustExec(`prepare stmt from 'select * from t where a IN (?,?,?)'`) + tk.MustExec(`set @a=1,@b=3,@c=4`) + tk.MustQuery(`execute stmt using @a,@b,@c`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BC 3")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`execute stmt using @a,@b,@c`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BC 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).CheckAt([]int{0}, [][]any{ + {"IndexLookUp_7"}, + {"├─IndexRangeScan_5(Build)"}, + {"└─TableRowIDScan_6(Probe)"}}) + tk.MustExec(`set @a=2,@b=5,@c=4`) + tk.MustQuery(`execute stmt using @a,@b,@c`).Sort().Check(testkit.Rows("AC 4", "BA 5", "abc 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) +} + +func TestNonPreparedPlanCachePartitionIndex(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set @@tidb_enable_non_prepared_plan_cache=1`) + tk.MustExec("use test") + tk.MustExec(`create table t (b varchar(255), a int primary key nonclustered, key (b)) partition by key(a) partitions 3`) + // [Batch]PointGet does not use the plan cache, + // since it is already using the fast path! + tk.MustExec(`insert into t values ('Ab', 1),('abc',2),('BC',3),('AC',4),('BA',5),('cda',6)`) + tk.MustExec(`analyze table t`) + tk.MustQuery(`explain format='plan_cache' select * from t where a IN (2,1,4,1,1,5,5)`).Check(testkit.Rows(""+ + "IndexLookUp_7 4.00 root partition:p1,p2 ", + "├─IndexRangeScan_5(Build) 4.00 cop[tikv] table:t, index:PRIMARY(a) range:[1,1], [2,2], [4,4], [5,5], keep order:false", + "└─TableRowIDScan_6(Probe) 4.00 cop[tikv] table:t keep order:false")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a IN (2,1,4,1,1,5,5)`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BA 5", "abc 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a IN (1,3,4)`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BC 3")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a IN (1,3,4)`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BC 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a IN (2,5,4,2,5,5,1)`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BA 5", "abc 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a IN (1,2,3,4,5,5,1)`).Sort().Check(testkit.Rows("AC 4", "Ab 1", "BA 5", "BC 3", "abc 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select count(*) from t partition (p0)`).Check(testkit.Rows("0")) + tk.MustQuery(`select count(*) from t partition (p1)`).Check(testkit.Rows("5")) + tk.MustQuery(`select * from t partition (p2)`).Check(testkit.Rows("Ab 1")) + + tk.MustQuery(`explain format='plan_cache' select * from t where a = 2`).Check(testkit.Rows("Point_Get_1 1.00 root table:t, partition:p1, index:PRIMARY(a) ")) + tk.MustQuery(`explain format='plan_cache' select * from t where a = 2`).Check(testkit.Rows("Point_Get_1 1.00 root table:t, partition:p1, index:PRIMARY(a) ")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`select * from t where a = 2`).Check(testkit.Rows("abc 2")) + tk.MustExec(`create table tk (a int primary key nonclustered, b varchar(255), key (b)) partition by key (a) partitions 3`) + tk.MustExec(`insert into tk select a, b from t`) + tk.MustExec(`analyze table tk`) + tk.MustQuery(`explain format='plan_cache' select * from tk where a = 2`).Check(testkit.Rows("Point_Get_1 1.00 root table:tk, partition:p1, index:PRIMARY(a) ")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + // PointGet will use Fast Plan, so no Plan Cache, even for Key Partitioned tables. + tk.MustQuery(`select * from tk where a = 2`).Check(testkit.Rows("2 abc")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) +} + +func TestFixControl33031(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustQuery(`select @@session.tidb_enable_prepared_plan_cache`).Check(testkit.Rows("1")) + + tk.MustExec("use test") + tk.MustExec(`drop table if exists t`) + tk.MustExec(`CREATE TABLE t (a int primary key, b varchar(255), key (b)) PARTITION BY HASH (a) partitions 5`) + tk.MustExec(`insert into t values(0,0),(1,1),(2,2),(3,3),(4,4)`) + tk.MustExec(`insert into t select a + 5, b + 5 from t`) + tk.MustExec(`analyze table t`) + + tk.MustExec(`prepare stmt from 'select * from t where a = ?'`) + tk.MustExec(`set @a = 2`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("2 2")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @a = 3`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("3 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @@tidb_opt_fix_control = "33031:ON"`) + tk.MustExec(`set @a = 1`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("1 1")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip plan-cache: plan rebuild failed, Fix33031 fix-control set and partitioned table in cached Point Get plan")) + tk.MustExec(`set @@tidb_opt_fix_control = "33031:OFF"`) + tk.MustExec(`set @a = 2`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("2 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`deallocate prepare stmt`) + + tk.MustExec(`prepare stmt from 'select * from t where a IN (?,?)'`) + tk.MustExec(`set @a = 2, @b = 5`) + tk.MustQuery(`execute stmt using @a, @b`).Sort().Check(testkit.Rows("2 2", "5 5")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @a = 3, @b = 0`) + tk.MustQuery(`execute stmt using @a, @b`).Sort().Check(testkit.Rows("0 0", "3 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @@tidb_opt_fix_control = "33031:ON"`) + tk.MustExec(`set @a = 1, @b = 2`) + tk.MustQuery(`execute stmt using @a, @b`).Check(testkit.Rows("1 1", "2 2")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip plan-cache: plan rebuild failed, Fix33031 fix-control set and partitioned table in cached Batch Point Get plan")) + tk.MustExec(`set @@tidb_opt_fix_control = "33031:OFF"`) + tk.MustExec(`set @a = 2, @b = 3`) + tk.MustQuery(`execute stmt using @a, @b`).Check(testkit.Rows("2 2", "3 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) +} + +func TestPlanCachePartitionDuplicates(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int unique key, b int) partition by range (a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + tk.MustExec(`insert into t values (1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7)`) + tk.MustExec(`insert into t select a + 10000, b + 10000 from t`) + tk.MustExec(`insert into t select a + 20000, b + 20000 from t`) + tk.MustExec(`analyze table t`) + tk.MustQuery(`select @@session.tidb_enable_prepared_plan_cache`).Check(testkit.Rows("1")) + tk.MustExec(`prepare stmt from 'select * from t use index(a) where a in (?,?,?)'`) + tk.MustExec(`set @a0 = 1, @a1 = 10001, @a2 = 2`) + tk.MustQuery(`execute stmt using @a0, @a1, @a2`).Sort().Check(testkit.Rows("1 1", "10001 10001", "2 2")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @a0 = 3, @a1 = 20001, @a2 = 50000`) + tk.MustQuery(`execute stmt using @a0, @a1, @a2`).Sort().Check(testkit.Rows("20001 20001", "3 3")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).CheckAt([]int{0}, [][]any{{"Batch_Point_Get_1"}}) + tk.MustExec(`set @a0 = 30003, @a1 = 20002, @a2 = 4`) + tk.MustQuery(`execute stmt using @a0, @a1, @a2`).Sort().Check(testkit.Rows("20002 20002", "30003 30003", "4 4")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tkExplain := testkit.NewTestKit(t, store) + tkExplain.MustExec(`use test`) + tkExplain.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).CheckAt([]int{0}, [][]any{{"Batch_Point_Get_1"}}) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip plan-cache: plan rebuild failed, rebuild to get an unsafe range, IndexValue length diff")) +} + +func TestPreparedStmtIndexLookup(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int, b int, unique key (a)) +partition by hash (a) partitions 3`) + + for i := 0; i < 100; i++ { + tk.MustExec("insert into t values (?, ?)", i, i) + } + tk.MustExec("analyze table t") + tk.MustExec(`set tidb_partition_prune_mode = 'static'`) + tk.MustQuery(`select b from t where a = 1 or a = 10 or a = 10 or a = 999999`).Sort().Check(testkit.Rows("1", "10")) + tk.MustQuery(`explain format='brief' select b from t where a = 1 or a = 10 or a = 10 or a = 999999`).Check(testkit.Rows(""+ + "PartitionUnion 6.00 root ", + "├─Projection 3.00 root test.t.b", + "│ └─Batch_Point_Get 3.00 root table:t, partition:p0, index:a(a) keep order:false, desc:false", + "└─Projection 3.00 root test.t.b", + " └─Batch_Point_Get 3.00 root table:t, partition:p1, index:a(a) keep order:false, desc:false")) + tk.MustExec(`prepare stmt from 'select b from t where a = 1 or a = 10 or a = 10 or a = 999999'`) + tk.MustQuery(`execute stmt`) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).MultiCheckContain([]string{"PartitionUnion", "Batch_Point_Get", "partition:p0", "partition:p1"}) + tk.MustQuery(`execute stmt`) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip prepared plan-cache: query accesses partitioned tables is un-cacheable if tidb_partition_pruning_mode = 'static'")) +} + +func TestIndexRange(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + + tk.MustExec(`CREATE TABLE t0 (id bigint NOT NULL AUTO_INCREMENT PRIMARY KEY)`) + tk.MustExec(`CREATE TABLE t1(c0 FLOAT ZEROFILL, PRIMARY KEY(c0));`) + tk.MustExec(`INSERT INTO t0 (id) VALUES (1),(2),(3),(4),(5),(6),(7),(8),(9),(10),(11);`) + tk.MustExec("INSERT INTO t1(c0) VALUES (1);") + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1;`) + tk.MustQuery(`SELECT t0.* FROM t0 WHERE (id = 1 or id = 9223372036854775808);`).Check(testkit.Rows("1")) + tk.MustQuery("SELECT t1.c0 FROM t1 WHERE t1.c0!=BIN(-1);").Check(testkit.Rows("1")) +} + +func TestPlanCacheDirtyTables(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + + for _, t1Dirty := range []bool{true, false} { + for _, t2Dirty := range []bool{true, false} { + tk.MustExec(`create table t1 (a int);`) + tk.MustExec(`create table t2 (a int);`) + tk.MustExec(`begin`) + tk.MustExec(`prepare st from 'select 1 from t1, t2'`) + if t1Dirty { + tk.MustExec(`insert into t1 values (1)`) + } + if t2Dirty { + tk.MustExec(`insert into t2 values (1)`) + } + tk.MustExec(`execute st`) // generate a cached plan with t1Dirty & t2Dirty + tk.MustExec(`commit`) + + // test cases + for _, testT1Dirty := range []bool{true, false} { + for _, testT2Dirty := range []bool{true, false} { + tk.MustExec(`begin`) + if testT1Dirty { + tk.MustExec(`insert into t1 values (1)`) + } + if testT2Dirty { + tk.MustExec(`insert into t2 values (1)`) + } + tk.MustExec(`execute st`) + + if testT1Dirty == t1Dirty && testT2Dirty == t2Dirty { + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + } else { + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + } + + tk.MustExec(`commit`) + } + } + tk.MustExec(`drop table t1, t2`) + } + } +} + +func TestInstancePlanCacheAcrossSession(t *testing.T) { + store := testkit.CreateMockStore(t) + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec(`use test`) + tk1.MustExec(`create table t (a int)`) + tk1.MustExec(`insert into t values (1), (2), (3), (4), (5)`) + tk1.Session().GetSessionVars().EnableInstancePlanCache = true + tk1.MustExec(`prepare st from 'select a from t where a < ?'`) + tk1.MustExec(`set @a=2`) + tk1.MustQuery(`execute st using @a`).Sort().Check(testkit.Rows(`1`)) + tk1.MustExec(`set @a=3`) + tk1.MustQuery(`execute st using @a`).Sort().Check(testkit.Rows(`1`, `2`)) + tk1.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + // session2 can share session1's cached plan + tk2 := testkit.NewTestKit(t, store) + tk2.Session().GetSessionVars().EnableInstancePlanCache = true + tk2.MustExec(`use test`) + tk2.MustExec(`prepare st from 'select a from t where a < ?'`) + tk2.MustExec(`set @a=4`) + tk2.MustQuery(`execute st using @a`).Sort().Check(testkit.Rows(`1`, `2`, `3`)) + tk2.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) +} + +func TestIssue54652(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec(`use test`) + tk.MustExec(`create table t (pk int, a int, primary key(pk))`) + tk.MustExec(`set autocommit=on`) + tk.MustQuery(`select @@autocommit`).Check(testkit.Rows("1")) + tk.MustExec(`set @pk=1`) + + tk.MustExec(`prepare st from 'select * from t where pk=? for update'`) + tk.MustExec(`execute st using @pk`) + tk.MustExec(`execute st using @pk`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + tk.MustExec(`begin`) + tk.MustExec(`execute st using @pk`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) // can't reuse since it's in txn now. + tk.MustExec(`commit`) +} diff --git a/pkg/planner/core/plan_cache_utils.go b/pkg/planner/core/plan_cache_utils.go new file mode 100644 index 0000000000000..3a1b61ec1aeba --- /dev/null +++ b/pkg/planner/core/plan_cache_utils.go @@ -0,0 +1,737 @@ +// Copyright 2017 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "cmp" + "context" + "math" + "slices" + "sort" + "strconv" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/bindinfo" + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/util" + "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/table" + "github.com/pingcap/tidb/pkg/types" + driver "github.com/pingcap/tidb/pkg/types/parser_driver" + "github.com/pingcap/tidb/pkg/util/codec" + "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" + "github.com/pingcap/tidb/pkg/util/hack" + "github.com/pingcap/tidb/pkg/util/hint" + "github.com/pingcap/tidb/pkg/util/intest" + "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/pingcap/tidb/pkg/util/size" + atomic2 "go.uber.org/atomic" + "go.uber.org/zap" +) + +const ( + // MaxCacheableLimitCount is the max limit count for cacheable query. + MaxCacheableLimitCount = 10000 +) + +var ( + // PreparedPlanCacheMaxMemory stores the max memory size defined in the global config "performance-server-memory-quota". + PreparedPlanCacheMaxMemory = *atomic2.NewUint64(math.MaxUint64) +) + +type paramMarkerExtractor struct { + markers []ast.ParamMarkerExpr +} + +func (*paramMarkerExtractor) Enter(in ast.Node) (ast.Node, bool) { + return in, false +} + +func (e *paramMarkerExtractor) Leave(in ast.Node) (ast.Node, bool) { + if x, ok := in.(*driver.ParamMarkerExpr); ok { + e.markers = append(e.markers, x) + } + return in, true +} + +// GeneratePlanCacheStmtWithAST generates the PlanCacheStmt structure for this AST. +// paramSQL is the corresponding parameterized sql like 'select * from t where a?'. +// paramStmt is the Node of paramSQL. +func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, isPrepStmt bool, + paramSQL string, paramStmt ast.StmtNode, is infoschema.InfoSchema) (*PlanCacheStmt, base.Plan, int, error) { + vars := sctx.GetSessionVars() + var extractor paramMarkerExtractor + paramStmt.Accept(&extractor) + + // DDL Statements can not accept parameters + if _, ok := paramStmt.(ast.DDLNode); ok && len(extractor.markers) > 0 { + return nil, nil, 0, plannererrors.ErrPrepareDDL + } + + switch stmt := paramStmt.(type) { + case *ast.ImportIntoStmt, *ast.LoadDataStmt, *ast.PrepareStmt, *ast.ExecuteStmt, *ast.DeallocateStmt, *ast.NonTransactionalDMLStmt: + return nil, nil, 0, plannererrors.ErrUnsupportedPs + case *ast.SelectStmt: + if stmt.SelectIntoOpt != nil { + return nil, nil, 0, plannererrors.ErrUnsupportedPs + } + } + + // Prepare parameters should NOT over 2 bytes(MaxUint16) + // https://dev.mysql.com/doc/internals/en/com-stmt-prepare-response.html#packet-COM_STMT_PREPARE_OK. + if len(extractor.markers) > math.MaxUint16 { + return nil, nil, 0, plannererrors.ErrPsManyParam + } + + ret := &PreprocessorReturn{InfoSchema: is} // is can be nil, and + err := Preprocess(ctx, sctx, paramStmt, InPrepare, WithPreprocessorReturn(ret)) + if err != nil { + return nil, nil, 0, err + } + + // The parameter markers are appended in visiting order, which may not + // be the same as the position order in the query string. We need to + // sort it by position. + slices.SortFunc(extractor.markers, func(i, j ast.ParamMarkerExpr) int { + return cmp.Compare(i.(*driver.ParamMarkerExpr).Offset, j.(*driver.ParamMarkerExpr).Offset) + }) + paramCount := len(extractor.markers) + for i := 0; i < paramCount; i++ { + extractor.markers[i].SetOrder(i) + } + + prepared := &ast.Prepared{ + Stmt: paramStmt, + StmtType: ast.GetStmtLabel(paramStmt), + } + normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) + + var ( + cacheable bool + reason string + ) + if (isPrepStmt && !vars.EnablePreparedPlanCache) || // prepared statement + (!isPrepStmt && !vars.EnableNonPreparedPlanCache) { // non-prepared statement + cacheable = false + reason = "plan cache is disabled" + } else { + if isPrepStmt { + cacheable, reason = IsASTCacheable(ctx, sctx.GetPlanCtx(), paramStmt, ret.InfoSchema) + } else { + cacheable = true // it is already checked here + } + + if !cacheable && fixcontrol.GetBoolWithDefault(vars.OptimizerFixControl, fixcontrol.Fix49736, false) { + sctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("force plan-cache: may use risky cached plan: %s", reason)) + cacheable = true + reason = "" + } + + if !cacheable { + sctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("skip prepared plan-cache: " + reason)) + } + } + + // For prepared statements like `prepare st from 'select * from t where a 0 { + // dynamic prune mode is not used, could be that global statistics not yet available! + cacheable = false + reason = "static partition prune mode used" + sctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("skip prepared plan-cache: " + reason)) + } + + // Collect information for metadata lock. + dbName := make([]model.CIStr, 0, len(vars.StmtCtx.MDLRelatedTableIDs)) + tbls := make([]table.Table, 0, len(vars.StmtCtx.MDLRelatedTableIDs)) + relateVersion := make(map[int64]uint64, len(vars.StmtCtx.MDLRelatedTableIDs)) + for id := range vars.StmtCtx.MDLRelatedTableIDs { + tbl, ok := is.TableByID(id) + if !ok { + logutil.BgLogger().Error("table not found in info schema", zap.Int64("tableID", id)) + return nil, nil, 0, errors.New("table not found in info schema") + } + db, ok := is.SchemaByID(tbl.Meta().DBID) + if !ok { + logutil.BgLogger().Error("database not found in info schema", zap.Int64("dbID", tbl.Meta().DBID)) + return nil, nil, 0, errors.New("database not found in info schema") + } + dbName = append(dbName, db.Name) + tbls = append(tbls, tbl) + relateVersion[id] = tbl.Meta().Revision + } + + preparedObj := &PlanCacheStmt{ + PreparedAst: prepared, + StmtDB: vars.CurrentDB, + StmtText: paramSQL, + VisitInfos: destBuilder.GetVisitInfo(), + NormalizedSQL: normalizedSQL, + SQLDigest: digest, + ForUpdateRead: destBuilder.GetIsForUpdateRead(), + SnapshotTSEvaluator: ret.SnapshotTSEvaluator, + StmtCacheable: cacheable, + UncacheableReason: reason, + dbName: dbName, + tbls: tbls, + SchemaVersion: ret.InfoSchema.SchemaMetaVersion(), + RelateVersion: relateVersion, + Params: extractor.markers, + } + + stmtProcessor := &planCacheStmtProcessor{ctx: ctx, is: is, stmt: preparedObj} + paramStmt.Accept(stmtProcessor) + + if err = checkPreparedPriv(ctx, sctx, preparedObj, ret.InfoSchema); err != nil { + return nil, nil, 0, err + } + return preparedObj, p, paramCount, nil +} + +func hashInt64Uint64Map(b []byte, m map[int64]uint64) []byte { + keys := make([]int64, 0, len(m)) + for k := range m { + keys = append(keys, k) + } + sort.Slice(keys, func(i, j int) bool { + return keys[i] < keys[j] + }) + + for _, k := range keys { + v := m[k] + b = codec.EncodeInt(b, k) + b = codec.EncodeUint(b, v) + } + return b +} + +// NewPlanCacheKey creates the plan cache key for this statement. +// Note: lastUpdatedSchemaVersion will only be set in the case of rc or for update read in order to +// differentiate the cache key. In other cases, it will be 0. +// All information that might affect the plan should be considered in this function. +func NewPlanCacheKey(sctx sessionctx.Context, stmt *PlanCacheStmt) (key, binding string, cacheable bool, reason string, err error) { + binding, ignored := bindinfo.MatchSQLBindingForPlanCache(sctx, stmt.PreparedAst.Stmt, &stmt.BindingInfo) + if ignored { + return "", binding, false, "ignore plan cache by binding", nil + } + + // In rc or for update read, we need the latest schema version to decide whether we need to + // rebuild the plan. So we set this value in rc or for update read. In other cases, let it be 0. + var latestSchemaVersion int64 + if sctx.GetSessionVars().IsIsolation(ast.ReadCommitted) || stmt.ForUpdateRead { + // In Rc or ForUpdateRead, we should check if the information schema has been changed since + // last time. If it changed, we should rebuild the plan. Here, we use a different and more + // up-to-date schema version which can lead plan cache miss and thus, the plan will be rebuilt. + latestSchemaVersion = domain.GetDomain(sctx).InfoSchema().SchemaMetaVersion() + } + + // rebuild key to exclude kv.TiFlash when stmt is not read only + vars := sctx.GetSessionVars() + if _, isolationReadContainTiFlash := vars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmt.PreparedAst.Stmt, vars) { + delete(vars.IsolationReadEngines, kv.TiFlash) + defer func() { + vars.IsolationReadEngines[kv.TiFlash] = struct{}{} + }() + } + + if stmt.StmtText == "" { + return "", "", false, "", errors.New("no statement text") + } + if stmt.SchemaVersion == 0 && !intest.InTest { + return "", "", false, "", errors.New("Schema version uninitialized") + } + stmtDB := stmt.StmtDB + if stmtDB == "" { + stmtDB = vars.CurrentDB + } + timezoneOffset := 0 + if vars.TimeZone != nil { + _, timezoneOffset = time.Now().In(vars.TimeZone).Zone() + } + _, connCollation := vars.GetCharsetInfo() + + hash := make([]byte, 0, len(stmt.StmtText)*2) // TODO: a Pool for this + hash = append(hash, hack.Slice(stmtDB)...) + hash = append(hash, hack.Slice(stmt.StmtText)...) + hash = codec.EncodeInt(hash, stmt.SchemaVersion) + hash = hashInt64Uint64Map(hash, stmt.RelateVersion) + // Only be set in rc or for update read and leave it default otherwise. + // In Rc or ForUpdateRead, we should check whether the information schema has been changed when using plan cache. + // If it changed, we should rebuild the plan. lastUpdatedSchemaVersion help us to decide whether we should rebuild + // the plan in rc or for update read. + hash = codec.EncodeInt(hash, latestSchemaVersion) + hash = codec.EncodeInt(hash, int64(vars.SQLMode)) + hash = codec.EncodeInt(hash, int64(timezoneOffset)) + if _, ok := vars.IsolationReadEngines[kv.TiDB]; ok { + hash = append(hash, kv.TiDB.Name()...) + } + if _, ok := vars.IsolationReadEngines[kv.TiKV]; ok { + hash = append(hash, kv.TiKV.Name()...) + } + if _, ok := vars.IsolationReadEngines[kv.TiFlash]; ok { + hash = append(hash, kv.TiFlash.Name()...) + } + hash = codec.EncodeInt(hash, int64(vars.SelectLimit)) + hash = append(hash, hack.Slice(binding)...) + hash = append(hash, hack.Slice(connCollation)...) + hash = append(hash, hack.Slice(strconv.FormatBool(vars.InRestrictedSQL))...) + hash = append(hash, hack.Slice(strconv.FormatBool(variable.RestrictedReadOnly.Load()))...) + hash = append(hash, hack.Slice(strconv.FormatBool(variable.VarTiDBSuperReadOnly.Load()))...) + // expr-pushdown-blacklist can affect query optimization, so we need to consider it in plan cache. + hash = codec.EncodeInt(hash, expression.ExprPushDownBlackListReloadTimeStamp.Load()) + + // whether this query has sub-query + if stmt.hasSubquery { + if !vars.EnablePlanCacheForSubquery { + return "", "", false, "the switch 'tidb_enable_plan_cache_for_subquery' is off", nil + } + hash = append(hash, '1') + } else { + hash = append(hash, '0') + } + + // this variable might affect the plan + hash = append(hash, bool2Byte(vars.ForeignKeyChecks)) + + // "limit ?" can affect the cached plan: "limit 1" and "limit 10000" should use different plans. + if len(stmt.limits) > 0 { + if !vars.EnablePlanCacheForParamLimit { + return "", "", false, "the switch 'tidb_enable_plan_cache_for_param_limit' is off", nil + } + hash = append(hash, '|') + for _, node := range stmt.limits { + for _, valNode := range []ast.ExprNode{node.Count, node.Offset} { + if valNode == nil { + continue + } + if param, isParam := valNode.(*driver.ParamMarkerExpr); isParam { + typeExpected, val := CheckParamTypeInt64orUint64(param) + if !typeExpected { + return "", "", false, "unexpected value after LIMIT", nil + } + if val > MaxCacheableLimitCount { + return "", "", false, "limit count is too large", nil + } + hash = codec.EncodeUint(hash, val) + } + } + } + hash = append(hash, '|') + } + + // stats ver can affect cached plan + if sctx.GetSessionVars().PlanCacheInvalidationOnFreshStats { + var statsVerHash uint64 + for _, t := range stmt.tables { + statsVerHash += getLatestVersionFromStatsTable(sctx, t.Meta(), t.Meta().ID) // use '+' as the hash function for simplicity + } + hash = codec.EncodeUint(hash, statsVerHash) + } + + // handle dirty tables + dirtyTables := vars.StmtCtx.TblInfo2UnionScan + if len(dirtyTables) > 0 { + dirtyTableIDs := make([]int64, 0, len(dirtyTables)) // TODO: a Pool for this + for t, dirty := range dirtyTables { + if !dirty { + continue + } + dirtyTableIDs = append(dirtyTableIDs, t.ID) + } + sort.Slice(dirtyTableIDs, func(i, j int) bool { return dirtyTableIDs[i] < dirtyTableIDs[j] }) + for _, id := range dirtyTableIDs { + hash = codec.EncodeInt(hash, id) + } + } + + // txn status + hash = append(hash, '|') + hash = append(hash, bool2Byte(vars.InTxn())) + hash = append(hash, bool2Byte(vars.IsAutocommit())) + hash = append(hash, bool2Byte(config.GetGlobalConfig().PessimisticTxn.PessimisticAutoCommit.Load())) + + return string(hash), binding, true, "", nil +} + +func bool2Byte(flag bool) byte { + if flag { + return '1' + } + return '0' +} + +// PlanCacheValue stores the cached Statement and StmtNode. +type PlanCacheValue struct { + Plan base.Plan // not-read-only, session might update it before reusing + OutputColumns types.NameSlice // read-only + memoryUsage int64 // read-only + testKey int64 // test-only + paramTypes []*types.FieldType // read-only, all parameters' types, different parameters may share same plan + stmtHints *hint.StmtHints // read-only, hints which set session variables +} + +// CloneForInstancePlanCache clones a PlanCacheValue for instance plan cache. +// Since PlanCacheValue.Plan is not read-only, to solve the concurrency problem when sharing the same PlanCacheValue +// across multiple sessions, we need to clone the PlanCacheValue for each session. +func (v *PlanCacheValue) CloneForInstancePlanCache(ctx context.Context, newCtx base.PlanContext) (*PlanCacheValue, bool) { + phyPlan, ok := v.Plan.(base.PhysicalPlan) + if !ok { + return nil, false + } + clonedPlan, err := phyPlan.Clone(newCtx) + if err != nil { + return nil, false + } + if intest.InTest && ctx.Value(PlanCacheKeyTestClone{}) != nil { + ctx.Value(PlanCacheKeyTestClone{}).(func(plan, cloned base.Plan))(phyPlan, clonedPlan) + } + cloned := new(PlanCacheValue) + *cloned = *v + cloned.Plan = clonedPlan + return cloned, true +} + +// unKnownMemoryUsage represent the memory usage of uncounted structure, maybe need implement later +// 100 KiB is approximate consumption of a plan from our internal tests +const unKnownMemoryUsage = int64(50 * size.KB) + +// MemoryUsage return the memory usage of PlanCacheValue +func (v *PlanCacheValue) MemoryUsage() (sum int64) { + if v == nil { + return + } + + if v.memoryUsage > 0 { + return v.memoryUsage + } + switch x := v.Plan.(type) { + case base.PhysicalPlan: + sum = x.MemoryUsage() + case *Insert: + sum = x.MemoryUsage() + case *Update: + sum = x.MemoryUsage() + case *Delete: + sum = x.MemoryUsage() + default: + sum = unKnownMemoryUsage + } + + sum += size.SizeOfInterface + size.SizeOfSlice*2 + int64(cap(v.OutputColumns))*size.SizeOfPointer + + size.SizeOfMap + size.SizeOfInt64*2 + if v.paramTypes != nil { + sum += int64(cap(v.paramTypes)) * size.SizeOfPointer + for _, ft := range v.paramTypes { + sum += ft.MemoryUsage() + } + } + + for _, name := range v.OutputColumns { + sum += name.MemoryUsage() + } + v.memoryUsage = sum + return +} + +// NewPlanCacheValue creates a SQLCacheValue. +func NewPlanCacheValue(plan base.Plan, names []*types.FieldName, + paramTypes []*types.FieldType, stmtHints *hint.StmtHints) *PlanCacheValue { + userParamTypes := make([]*types.FieldType, len(paramTypes)) + for i, tp := range paramTypes { + userParamTypes[i] = tp.Clone() + } + return &PlanCacheValue{ + Plan: plan, + OutputColumns: names, + paramTypes: userParamTypes, + stmtHints: stmtHints.Clone(), + } +} + +// planCacheStmtProcessor records all query features which may affect plan selection. +type planCacheStmtProcessor struct { + ctx context.Context + is infoschema.InfoSchema + stmt *PlanCacheStmt +} + +// Enter implements Visitor interface. +func (f *planCacheStmtProcessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { + switch node := in.(type) { + case *ast.Limit: + f.stmt.limits = append(f.stmt.limits, node) + case *ast.SubqueryExpr, *ast.ExistsSubqueryExpr: + f.stmt.hasSubquery = true + case *ast.TableName: + t, err := f.is.TableByName(f.ctx, node.Schema, node.Name) + if err == nil { + f.stmt.tables = append(f.stmt.tables, t) + } + } + return in, false +} + +// Leave implements Visitor interface. +func (*planCacheStmtProcessor) Leave(in ast.Node) (out ast.Node, ok bool) { + return in, true +} + +// PointGetExecutorCache caches the PointGetExecutor to further improve its performance. +// Don't forget to reset this executor when the prior plan is invalid. +type PointGetExecutorCache struct { + // Special (or tricky) optimization for PointGet Plan. + // Store the PointGet Plan in PlanCacheStmt directly to bypass the LRU Cache to gain some performance improvement. + // There is around 3% improvement, BenchmarkPreparedPointGet: 6450 ns/op --> 6250 ns/op. + pointPlan base.Plan + pointPlanHints *hint.StmtHints + columnNames types.NameSlice + + // the cache key for this statement, have to check whether the cache key changes before reusing this plan for safety. + planCacheKey string + + ColumnInfos any + // Executor is only used for point get scene. + // Notice that we should only cache the PointGetExecutor that have a snapshot with MaxTS in it. + // If the current plan is not PointGet or does not use MaxTS optimization, this value should be nil here. + Executor any +} + +// PlanCacheStmt store prepared ast from PrepareExec and other related fields +type PlanCacheStmt struct { + PreparedAst *ast.Prepared + StmtDB string // which DB the statement will be processed over + VisitInfos []visitInfo + Params []ast.ParamMarkerExpr + + PointGet PointGetExecutorCache + + // below fields are for PointGet short path + SchemaVersion int64 + + // RelateVersion stores the true cache plan table schema version, since each table schema can be updated separately in transaction. + RelateVersion map[int64]uint64 + + StmtCacheable bool // Whether this stmt is cacheable. + UncacheableReason string // Why this stmt is uncacheable. + + limits []*ast.Limit + hasSubquery bool + tables []table.Table // to capture table stats changes + + NormalizedSQL string + NormalizedPlan string + SQLDigest *parser.Digest + PlanDigest *parser.Digest + ForUpdateRead bool + SnapshotTSEvaluator func(sessionctx.Context) (uint64, error) + + BindingInfo bindinfo.BindingMatchInfo + + // the different between NormalizedSQL, NormalizedSQL4PC and StmtText: + // for the query `select * from t where a>1 and b ? and `b` < ? --> constants are normalized to '?', + // NormalizedSQL4PC: select * from `test` . `t` where `a` > ? and `b` < ? --> schema name is added, + // StmtText: select * from t where a>1 and b just format the original query; + StmtText string + + // dbName and tbls are used to add metadata lock. + dbName []model.CIStr + tbls []table.Table +} + +// GetPreparedStmt extract the prepared statement from the execute statement. +func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*PlanCacheStmt, error) { + if stmt.PrepStmt != nil { + return stmt.PrepStmt.(*PlanCacheStmt), nil + } + if stmt.Name != "" { + prepStmt, err := vars.GetPreparedStmtByName(stmt.Name) + if err != nil { + return nil, err + } + stmt.PrepStmt = prepStmt + return prepStmt.(*PlanCacheStmt), nil + } + return nil, plannererrors.ErrStmtNotFound +} + +// CheckTypesCompatibility4PC compares FieldSlice with []*types.FieldType +// Currently this is only used in plan cache to check whether the types of parameters are compatible. +// If the types of parameters are compatible, we can use the cached plan. +// tpsExpected is types from cached plan +func checkTypesCompatibility4PC(expected, actual any) bool { + if expected == nil || actual == nil { + return true // no need to compare types + } + tpsExpected := expected.([]*types.FieldType) + tpsActual := actual.([]*types.FieldType) + if len(tpsExpected) != len(tpsActual) { + return false + } + for i := range tpsActual { + // We only use part of logic of `func (ft *FieldType) Equal(other *FieldType)` here because (1) only numeric and + // string types will show up here, and (2) we don't need flen and decimal to be matched exactly to use plan cache + tpEqual := (tpsExpected[i].GetType() == tpsActual[i].GetType()) || + (tpsExpected[i].GetType() == mysql.TypeVarchar && tpsActual[i].GetType() == mysql.TypeVarString) || + (tpsExpected[i].GetType() == mysql.TypeVarString && tpsActual[i].GetType() == mysql.TypeVarchar) + if !tpEqual || tpsExpected[i].GetCharset() != tpsActual[i].GetCharset() || tpsExpected[i].GetCollate() != tpsActual[i].GetCollate() || + (tpsExpected[i].EvalType() == types.ETInt && mysql.HasUnsignedFlag(tpsExpected[i].GetFlag()) != mysql.HasUnsignedFlag(tpsActual[i].GetFlag())) { + return false + } + // When the type is decimal, we should compare the Flen and Decimal. + // We can only use the plan when both Flen and Decimal should less equal than the cached one. + // We assume here that there is no correctness problem when the precision of the parameters is less than the precision of the parameters in the cache. + if tpEqual && tpsExpected[i].GetType() == mysql.TypeNewDecimal && !(tpsExpected[i].GetFlen() >= tpsActual[i].GetFlen() && tpsExpected[i].GetDecimal() >= tpsActual[i].GetDecimal()) { + return false + } + } + return true +} + +func isSafePointGetPath4PlanCache(sctx base.PlanContext, path *util.AccessPath) bool { + // PointGet might contain some over-optimized assumptions, like `a>=1 and a<=1` --> `a=1`, but + // these assumptions may be broken after parameters change. + + if isSafePointGetPath4PlanCacheScenario1(path) { + return true + } + + // TODO: enable this fix control switch by default after more test cases are added. + if sctx != nil && sctx.GetSessionVars() != nil && sctx.GetSessionVars().OptimizerFixControl != nil { + fixControlOK := fixcontrol.GetBoolWithDefault(sctx.GetSessionVars().GetOptimizerFixControlMap(), fixcontrol.Fix44830, false) + if fixControlOK && (isSafePointGetPath4PlanCacheScenario2(path) || isSafePointGetPath4PlanCacheScenario3(path)) { + return true + } + } + + return false +} + +func isSafePointGetPath4PlanCacheScenario1(path *util.AccessPath) bool { + // safe scenario 1: each column corresponds to a single EQ, `a=1 and b=2 and c=3` --> `[1, 2, 3]` + if len(path.Ranges) <= 0 || path.Ranges[0].Width() != len(path.AccessConds) { + return false + } + for _, accessCond := range path.AccessConds { + f, ok := accessCond.(*expression.ScalarFunction) + if !ok || f.FuncName.L != ast.EQ { // column = constant + return false + } + } + return true +} + +func isSafePointGetPath4PlanCacheScenario2(path *util.AccessPath) bool { + // safe scenario 2: this Batch or PointGet is simply from a single IN predicate, `key in (...)` + if len(path.Ranges) <= 0 || len(path.AccessConds) != 1 { + return false + } + f, ok := path.AccessConds[0].(*expression.ScalarFunction) + if !ok || f.FuncName.L != ast.In { + return false + } + return len(path.Ranges) == len(f.GetArgs())-1 // no duplicated values in this in-list for safety. +} + +func isSafePointGetPath4PlanCacheScenario3(path *util.AccessPath) bool { + // safe scenario 3: this Batch or PointGet is simply from a simple DNF like `key=? or key=? or key=?` + if len(path.Ranges) <= 0 || len(path.AccessConds) != 1 { + return false + } + f, ok := path.AccessConds[0].(*expression.ScalarFunction) + if !ok || f.FuncName.L != ast.LogicOr { + return false + } + + dnfExprs := expression.FlattenDNFConditions(f) + if len(path.Ranges) != len(dnfExprs) { + // no duplicated values in this in-list for safety. + // e.g. `k=1 or k=2 or k=1` --> [[1, 1], [2, 2]] + return false + } + + for _, expr := range dnfExprs { + f, ok := expr.(*expression.ScalarFunction) + if !ok { + return false + } + switch f.FuncName.L { + case ast.EQ: // (k=1 or k=2) --> [k=1, k=2] + case ast.LogicAnd: // ((k1=1 and k2=1) or (k1=2 and k2=2)) --> [k1=1 and k2=1, k2=2 and k2=2] + cnfExprs := expression.FlattenCNFConditions(f) + if path.Ranges[0].Width() != len(cnfExprs) { // not all key columns are specified + return false + } + for _, expr := range cnfExprs { // k1=1 and k2=1 + f, ok := expr.(*expression.ScalarFunction) + if !ok || f.FuncName.L != ast.EQ { + return false + } + } + default: + return false + } + } + return true +} + +// parseParamTypes get parameters' types in PREPARE statement +func parseParamTypes(sctx sessionctx.Context, params []expression.Expression) (paramTypes []*types.FieldType) { + ectx := sctx.GetExprCtx().GetEvalCtx() + paramTypes = make([]*types.FieldType, 0, len(params)) + for _, param := range params { + if c, ok := param.(*expression.Constant); ok { // from binary protocol + paramTypes = append(paramTypes, c.GetType(ectx)) + continue + } + + // from text protocol, there must be a GetVar function + name := param.(*expression.ScalarFunction).GetArgs()[0].StringWithCtx(ectx) + tp, ok := sctx.GetSessionVars().GetUserVarType(name) + if !ok { + tp = types.NewFieldType(mysql.TypeNull) + } + paramTypes = append(paramTypes, tp) + } + return +} diff --git a/pkg/planner/core/tests/prepare/prepare_test.go b/pkg/planner/core/tests/prepare/prepare_test.go new file mode 100644 index 0000000000000..77a40d3eb8915 --- /dev/null +++ b/pkg/planner/core/tests/prepare/prepare_test.go @@ -0,0 +1,1665 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package prepare_test + +import ( + "context" + "fmt" + "math" + "math/rand" + "strconv" + "strings" + "testing" + "time" + + "github.com/pingcap/tidb/pkg/errno" + "github.com/pingcap/tidb/pkg/executor" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/metrics" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/auth" + "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/util" + "github.com/pingcap/tidb/pkg/util/hint" + "github.com/prometheus/client_golang/prometheus" + dto "github.com/prometheus/client_model/go" + "github.com/stretchr/testify/require" +) + +func TestPointGetPreparedPlan4PlanCache(t *testing.T) { + store := testkit.CreateMockStore(t) + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk1.MustExec("drop database if exists ps_text") + defer tk1.MustExec("drop database if exists ps_text") + tk1.MustExec("create database ps_text") + tk1.MustExec("use ps_text") + + tk1.MustExec(`create table t (a int, b int, c int, + primary key k_a(a), + unique key k_b(b))`) + tk1.MustExec("insert into t values (1, 1, 1)") + tk1.MustExec("insert into t values (2, 2, 2)") + tk1.MustExec("insert into t values (3, 3, 3)") + + pspk1Id, _, _, err := tk1.Session().PrepareStmt("select * from t where a = ?") + require.NoError(t, err) + tk1.Session().GetSessionVars().PreparedStmts[pspk1Id].(*core.PlanCacheStmt).StmtCacheable = false + + ctx := context.Background() + // first time plan generated + _, err = tk1.Session().ExecutePreparedStmt(ctx, pspk1Id, expression.Args2Expressions4Test(0)) + require.NoError(t, err) + + // using the generated plan but with different params + _, err = tk1.Session().ExecutePreparedStmt(ctx, pspk1Id, expression.Args2Expressions4Test(nil)) + require.NoError(t, err) +} + +func TestRandomFlushPlanCache(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk2 := testkit.NewTestKit(t, store) + var err error + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1(id int, a int, b int, key(a))") + tk.MustExec("create table t2(id int, a int, b int, key(a))") + tk.MustExec("prepare stmt1 from 'SELECT * from t1,t2 where t1.id = t2.id';") + tk.MustExec("prepare stmt2 from 'SELECT * from t1';") + tk.MustExec("prepare stmt3 from 'SELECT * from t1 where id = 1';") + tk.MustExec("prepare stmt4 from 'SELECT * from t2';") + tk.MustExec("prepare stmt5 from 'SELECT * from t2 where id = 1';") + + tk2.MustExec("use test") + tk2.MustExec("prepare stmt1 from 'SELECT * from t1,t2 where t1.id = t2.id';") + tk2.MustExec("prepare stmt2 from 'SELECT * from t1';") + tk2.MustExec("prepare stmt3 from 'SELECT * from t1 where id = 1';") + tk2.MustExec("prepare stmt4 from 'SELECT * from t2';") + tk2.MustExec("prepare stmt5 from 'SELECT * from t2 where id = 1';") + + prepareNum := 5 + execStmts := make([]string, 0, prepareNum) + for i := 1; i <= prepareNum; i++ { + execStmt := fmt.Sprintf("execute stmt%d", i) + execStmts = append(execStmts, execStmt) + } + + for i := 0; i < 10; i++ { + // Warm up to make sure all the plans are in the cache. + for _, execStmt := range execStmts { + tk.MustExec(execStmt) + tk.MustExec(execStmt) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + + tk2.MustExec(execStmt) + tk2.MustExec(execStmt) + tk2.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + } + + for j := 0; j < 10; j++ { + session1PC, session2PC := "1", "1" + // random to flush the plan cache + randNum := rand.Intn(10) + if randNum == 0 { + session1PC, session2PC = "0", "0" + if j%2 == 0 { + err = tk.ExecToErr("admin flush instance plan_cache;") + } else { + err = tk2.ExecToErr("admin flush instance plan_cache;") + } + require.NoError(t, err) + } else if randNum == 1 { + session1PC = "0" + err = tk.ExecToErr("admin flush session plan_cache;") + require.NoError(t, err) + } else if randNum == 2 { + session2PC = "0" + err = tk2.ExecToErr("admin flush session plan_cache;") + require.NoError(t, err) + } + + for _, execStmt := range execStmts { + tk.MustExec(execStmt) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(session1PC)) + + tk2.MustExec(execStmt) + tk2.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(session2PC)) + } + } + + err = tk.ExecToErr("admin flush instance plan_cache;") + require.NoError(t, err) + } + + err = tk.ExecToErr("admin flush global plan_cache;") + require.EqualError(t, err, "Do not support the 'admin flush global scope.'") +} + +func TestPrepareCache(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key, b int, c int, index idx1(b, a), index idx2(b))") + tk.MustExec("insert into t values(1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 1, 2)") + tk.MustExec(`prepare stmt1 from "select * from t use index(idx1) where a = ? and b = ?"`) + tk.MustExec(`prepare stmt2 from "select a, b from t use index(idx2) where b = ?"`) + tk.MustExec(`prepare stmt3 from "select * from t where a = ?"`) + tk.MustExec("set @a=1, @b=1") + // When executing one statement at the first time, we don't use cache, so we need to execute it at least twice to test the cache. + tk.MustQuery("execute stmt1 using @a, @b").Check(testkit.Rows("1 1 1")) + tk.MustQuery("execute stmt1 using @a, @b").Check(testkit.Rows("1 1 1")) + tk.MustQuery("execute stmt2 using @b").Check(testkit.Rows("1 1", "6 1")) + tk.MustQuery("execute stmt2 using @b").Check(testkit.Rows("1 1", "6 1")) + tk.MustQuery("execute stmt3 using @a").Check(testkit.Rows("1 1 1")) + tk.MustQuery("execute stmt3 using @a").Check(testkit.Rows("1 1 1")) + tk.MustExec(`prepare stmt4 from "select * from t where a > ?"`) + tk.MustExec("set @a=3") + tk.MustQuery("execute stmt4 using @a").Check(testkit.Rows("4 4 4", "5 5 5", "6 1 2")) + tk.MustQuery("execute stmt4 using @a").Check(testkit.Rows("4 4 4", "5 5 5", "6 1 2")) + tk.MustExec(`prepare stmt5 from "select c from t order by c"`) + tk.MustQuery("execute stmt5").Check(testkit.Rows("1", "2", "2", "3", "4", "5")) + tk.MustQuery("execute stmt5").Check(testkit.Rows("1", "2", "2", "3", "4", "5")) + tk.MustExec(`prepare stmt6 from "select distinct a from t order by a"`) + tk.MustQuery("execute stmt6").Check(testkit.Rows("1", "2", "3", "4", "5", "6")) + tk.MustQuery("execute stmt6").Check(testkit.Rows("1", "2", "3", "4", "5", "6")) + + // test privilege change + rootSe := tk.Session() + tk.MustExec("drop table if exists tp") + tk.MustExec(`create table tp(c1 int, c2 int, primary key (c1))`) + tk.MustExec(`insert into tp values(1, 1), (2, 2), (3, 3)`) + + tk.MustExec(`create user 'u_tp'@'localhost'`) + tk.MustExec(`grant select on test.tp to u_tp@'localhost';`) + + // user u_tp + userSess := newSession(t, store, "test") + require.NoError(t, userSess.Auth(&auth.UserIdentity{Username: "u_tp", Hostname: "localhost"}, nil, nil, nil)) + mustExec(t, userSess, `prepare ps_stp_r from 'select * from tp where c1 > ?'`) + mustExec(t, userSess, `set @p2 = 2`) + tk.SetSession(userSess) + tk.MustQuery(`execute ps_stp_r using @p2`).Check(testkit.Rows("3 3")) + tk.MustQuery(`execute ps_stp_r using @p2`).Check(testkit.Rows("3 3")) + tk.MustQuery(`execute ps_stp_r using @p2`).Check(testkit.Rows("3 3")) + + // root revoke + tk.SetSession(rootSe) + tk.MustExec(`revoke all on test.tp from 'u_tp'@'localhost';`) + + // user u_tp + tk.SetSession(userSess) + _, err := tk.Exec(`execute ps_stp_r using @p2`) + require.Error(t, err) + + // grant again + tk.SetSession(rootSe) + tk.MustExec(`grant select on test.tp to u_tp@'localhost';`) + + // user u_tp + tk.SetSession(userSess) + tk.MustQuery(`execute ps_stp_r using @p2`).Check(testkit.Rows("3 3")) + tk.MustQuery(`execute ps_stp_r using @p2`).Check(testkit.Rows("3 3")) + + // restore + tk.SetSession(rootSe) + tk.MustExec("drop table if exists tp") + tk.MustExec(`DROP USER 'u_tp'@'localhost';`) +} + +// dtype: tinyint, unsigned, float, decimal, year +// rtype: null, valid, out-of-range, invalid, str, exists +func randValue(tk *testkit.TestKit, tbl, col, dtype, rtype string) string { + if rtype == "" { + rtypes := []string{"null", "valid", "out-of-range", "invalid", "str", "exists"} + rtype = rtypes[rand.Intn(len(rtypes))] + } + if rtype == "null" { + return "null" + } + if rtype == "exists" { + res := tk.MustQuery(fmt.Sprintf("select %v from %v limit 1", col, tbl)).Rows()[0][0].(string) + if res == "" { + res = "null" + } + return res + } + switch dtype { + case "tinyint": + switch rtype { + case "valid": + return fmt.Sprintf("%v", -128+rand.Intn(256)) + case "out-of-range": + return fmt.Sprintf("%v", 128+rand.Intn(1024)) + case "invalid": + return "'invalid-tinyint'" + case "str": + return fmt.Sprintf("'%v'", -128+rand.Intn(256)) + } + case "unsigned": + switch rtype { + case "valid": + return fmt.Sprintf("%v", rand.Intn(4294967295)) + case "out-of-range": + return fmt.Sprintf("-%v", rand.Intn(4294967295)) + case "invalid": + return "'invalid-unsigned-int'" + case "str": + return fmt.Sprintf("'%v'", rand.Intn(4294967295)) + } + case "float": + switch rtype { + case "valid": + return fmt.Sprintf("%v%.4fE%v", []string{"+", "-"}[rand.Intn(2)], rand.Float32(), rand.Intn(38)) + case "out-of-range": + return fmt.Sprintf("%v%.4fE%v", []string{"+", "-"}[rand.Intn(2)], rand.Float32(), rand.Intn(100)+38) + case "invalid": + return "'invalid-float'" + case "str": + return fmt.Sprintf("'%v%.4fE%v'", []string{"+", "-"}[rand.Intn(2)], rand.Float32(), rand.Intn(38)) + } + case "decimal": // (10,2) + switch rtype { + case "valid": + return fmt.Sprintf("%v%v.%v", []string{"+", "-"}[rand.Intn(2)], rand.Intn(99999999), rand.Intn(100)) + case "out-of-range": + switch rand.Intn(2) { + case 0: + return fmt.Sprintf("%v%v.%v", []string{"+", "-"}[rand.Intn(2)], rand.Intn(99999999), rand.Intn(100000)+100000) + case 1: + return fmt.Sprintf("%v%v.%v", []string{"+", "-"}[rand.Intn(2)], rand.Intn(99999999)+99999999+1, rand.Intn(100)) + } + case "invalid": + return "'invalid-decimal'" + case "str": + return fmt.Sprintf("'%v%v.%v'", []string{"+", "-"}[rand.Intn(2)], rand.Intn(99999999), rand.Intn(100)) + } + case "year": + switch rtype { + case "valid": + return fmt.Sprintf("%v", 1901+rand.Intn(2155-1901)) + case "out-of-range": + return fmt.Sprintf("%v", 2156+rand.Intn(2155-1901)) + case "invalid": + return "'invalid-year'" + case "str": + return fmt.Sprintf("'%v'", 1901+rand.Intn(2155-1901)) + } + } + return "'invalid-type-" + dtype + "'" +} + +func TestPrepareCacheChangingParamType(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists t_tinyint, t_unsigned, t_float, t_decimal, t_year`) + tk.MustExec(`create table t_tinyint (a tinyint, b tinyint, key(a))`) + tk.MustExec(`create table t_unsigned (a int unsigned, b int unsigned, key(a))`) + tk.MustExec(`create table t_float(a float, b float, key(a))`) + tk.MustExec(`create table t_decimal(a decimal(10,2), b decimal(10,2), key(a))`) + tk.MustExec(`create table t_year(a year, b year, key(a))`) + for _, dtype := range []string{"tinyint", "unsigned", "float", "decimal", "year"} { + tbl := "t_" + dtype + for i := 0; i < 10; i++ { + tk.MustExec(fmt.Sprintf("insert into %v values (%v, %v)", tbl, randValue(nil, "", "", dtype, "valid"), randValue(nil, "", "", dtype, "valid"))) + } + tk.MustExec(fmt.Sprintf("insert into %v values (null, null)", tbl)) + tk.MustExec(fmt.Sprintf("insert into %v values (%v, null)", tbl, randValue(nil, "", "", dtype, "valid"))) + tk.MustExec(fmt.Sprintf("insert into %v values (null, %v)", tbl, randValue(nil, "", "", dtype, "valid"))) + + for round := 0; round < 10; round++ { + tk.MustExec(fmt.Sprintf(`prepare s1 from 'select * from %v where a=?'`, tbl)) + tk.MustExec(fmt.Sprintf(`prepare s2 from 'select * from %v where b=?'`, tbl)) + tk.MustExec(fmt.Sprintf(`prepare s3 from 'select * from %v where a in (?, ?, ?)'`, tbl)) + tk.MustExec(fmt.Sprintf(`prepare s4 from 'select * from %v where b in (?, ?, ?)'`, tbl)) + tk.MustExec(fmt.Sprintf(`prepare s5 from 'select * from %v where a>?'`, tbl)) + tk.MustExec(fmt.Sprintf(`prepare s6 from 'select * from %v where b>?'`, tbl)) + tk.MustExec(fmt.Sprintf(`prepare s7 from 'select * from %v where a>? and b>?'`, tbl)) + + for query := 0; query < 10; query++ { + a1, a2, a3 := randValue(tk, tbl, "a", dtype, ""), randValue(tk, tbl, "a", dtype, ""), randValue(tk, tbl, "a", dtype, "") + b1, b2, b3 := randValue(tk, tbl, "b", dtype, ""), randValue(tk, tbl, "b", dtype, ""), randValue(tk, tbl, "b", dtype, "") + tk.MustExec(fmt.Sprintf(`set @a1=%v,@a2=%v,@a3=%v`, a1, a2, a3)) + tk.MustExec(fmt.Sprintf(`set @b1=%v,@b2=%v,@b3=%v`, b1, b2, b3)) + + compareResult := func(sql1, sql2 string) { + raw, err := tk.Exec(sql1) + if err != nil { + require.Error(t, tk.ExecToErr(sql2)) + return + } + rs := tk.ResultSetToResult(raw, fmt.Sprintf("sql1:%s, sql2:%v", sql1, sql2)) + rs.Sort().Check(tk.MustQuery(sql2).Sort().Rows()) + } + + compareResult(`execute s1 using @a1`, fmt.Sprintf(`select * from %v where a=%v`, tbl, a1)) + compareResult(`execute s2 using @b1`, fmt.Sprintf(`select * from %v where b=%v`, tbl, b1)) + compareResult(`execute s3 using @a1,@a2,@a3`, fmt.Sprintf(`select * from %v where a in (%v,%v,%v)`, tbl, a1, a2, a3)) + compareResult(`execute s4 using @b1,@b2,@b3`, fmt.Sprintf(`select * from %v where b in (%v,%v,%v)`, tbl, b1, b2, b3)) + compareResult(`execute s5 using @a1`, fmt.Sprintf(`select * from %v where a>%v`, tbl, a1)) + compareResult(`execute s6 using @b1`, fmt.Sprintf(`select * from %v where b>%v`, tbl, b1)) + compareResult(`execute s7 using @a1,@b1`, fmt.Sprintf(`select * from %v where a>%v and b>%v`, tbl, a1, b1)) + } + } + } +} + +func TestPrepareCacheDeferredFunction(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (id int PRIMARY KEY, c1 TIMESTAMP(3) NOT NULL DEFAULT '2019-01-14 10:43:20', KEY idx1 (c1))") + tk.MustExec("prepare sel1 from 'select id, c1 from t1 where c1 < now(3)'") + + sql1 := "execute sel1" + expectedPattern := `IndexReader\(Index\(t1.idx1\)\[\[-inf,[0-9]{4}-(0[1-9]|1[0-2])-(0[1-9]|[1-2][0-9]|3[0-1]) (2[0-3]|[01][0-9]):[0-5][0-9]:[0-5][0-9].[0-9][0-9][0-9]\)\]\)` + + var cnt [2]float64 + var planStr [2]string + metrics.ResettablePlanCacheCounterFortTest = true + metrics.PlanCacheCounter.Reset() + counter := metrics.PlanCacheCounter.WithLabelValues("prepare") + ctx := context.TODO() + p := parser.New() + p.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true}) + for i := 0; i < 2; i++ { + stmt, err := p.ParseOneStmt(sql1, "", "") + require.NoError(t, err) + is := tk.Session().GetInfoSchema().(infoschema.InfoSchema) + builder, _ := core.NewPlanBuilder().Init(tk.Session().GetPlanCtx(), is, hint.NewQBHintHandler(nil)) + p, err := builder.Build(ctx, stmt) + require.NoError(t, err) + execPlan, ok := p.(*core.Execute) + require.True(t, ok) + err = executor.ResetContextOfStmt(tk.Session(), stmt) + require.NoError(t, err) + plan, _, err := core.GetPlanFromPlanCache(ctx, tk.Session(), false, is, execPlan.PrepStmt, execPlan.Params) + require.NoError(t, err) + planStr[i] = core.ToString(plan) + require.Regexpf(t, expectedPattern, planStr[i], "for %dth %s", i, sql1) + pb := &dto.Metric{} + err = counter.Write(pb) + require.NoError(t, err) + cnt[i] = pb.GetCounter().GetValue() + require.Equal(t, float64(i), cnt[i]) + time.Sleep(time.Millisecond * 10) + } + require.Lessf(t, planStr[0], planStr[1], "plan 1: %v, plan 2: %v", planStr[0], planStr[1]) +} + +func TestPrepareCacheNow(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + + tk.MustExec("use test") + tk.MustExec(`prepare stmt1 from "select now(), current_timestamp(), utc_timestamp(), unix_timestamp(), sleep(0.1), now(), current_timestamp(), utc_timestamp(), unix_timestamp()"`) + // When executing one statement at the first time, we don't usTestPrepareCacheDeferredFunctione cache, so we need to execute it at least twice to test the cache. + _ = tk.MustQuery("execute stmt1").Rows() + rs := tk.MustQuery("execute stmt1").Rows() + require.Equal(t, rs[0][5].(string), rs[0][0].(string)) + require.Equal(t, rs[0][6].(string), rs[0][1].(string)) + require.Equal(t, rs[0][7].(string), rs[0][2].(string)) + require.Equal(t, rs[0][8].(string), rs[0][3].(string)) + + tk.MustExec("create table t (a int);") + tk.MustExec("insert into t values(1);") + tk.MustExec("set @@tidb_enable_prepared_plan_cache=0;") + tk.MustExec("set global tidb_sysdate_is_now=0;") + tk.MustExec("prepare s from 'select sleep(a), now(6), sysdate(6),sysdate(6)=now(6) from t';") + t1 := tk.MustQuery("execute s").Rows() + tk.MustExec("set global tidb_sysdate_is_now=1;") + t2 := tk.MustQuery("execute s").Rows() + require.NotEqual(t, t1, t2) +} + +func TestPrepareOverMaxPreparedStmtCount(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + // test prepare and deallocate. + prePrepared := readGaugeInt(metrics.PreparedStmtGauge) + tk.MustExec(`prepare stmt1 from "select 1"`) + onePrepared := readGaugeInt(metrics.PreparedStmtGauge) + require.Equal(t, onePrepared, prePrepared+1) + tk.MustExec(`deallocate prepare stmt1`) + deallocPrepared := readGaugeInt(metrics.PreparedStmtGauge) + require.Equal(t, deallocPrepared, prePrepared) + + // test change global limit and make it affected in test session. + tk.MustQuery("select @@max_prepared_stmt_count").Check(testkit.Rows("-1")) + tk.MustExec("set @@global.max_prepared_stmt_count = 2") + tk.MustQuery("select @@global.max_prepared_stmt_count").Check(testkit.Rows("2")) + + // test close session to give up all prepared stmt + tk.MustExec(`prepare stmt2 from "select 1"`) + prePrepared = readGaugeInt(metrics.PreparedStmtGauge) + tk.Session().Close() + drawPrepared := readGaugeInt(metrics.PreparedStmtGauge) + require.Equal(t, drawPrepared, prePrepared-1) + + // test meet max limit. + tk.RefreshSession() + tk.MustQuery("select @@max_prepared_stmt_count").Check(testkit.Rows("2")) + for i := 1; ; i++ { + prePrepared = readGaugeInt(metrics.PreparedStmtGauge) + if prePrepared >= 2 { + tk.MustGetErrCode(`prepare stmt`+strconv.Itoa(i)+` from "select 1"`, errno.ErrMaxPreparedStmtCountReached) + break + } + tk.MustExec(`prepare stmt` + strconv.Itoa(i) + ` from "select 1"`) + } +} + +// nolint:unused +func readGaugeInt(g prometheus.Gauge) int { + ch := make(chan prometheus.Metric, 1) + g.Collect(ch) + m := <-ch + mm := &dto.Metric{} + err := m.Write(mm) + if err != nil { + panic(err) + } + + return int(mm.GetGauge().GetValue()) +} + +func TestPrepareWithSnapshot(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + safePointName := "tikv_gc_safe_point" + safePointValue := "20060102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int primary key, v int)") + tk.MustExec("insert into t select 1, 2") + tk.MustExec("begin") + ts := tk.MustQuery("select @@tidb_current_ts").Rows()[0][0].(string) + tk.MustExec("commit") + tk.MustExec("update t set v = 3 where id = 1") + tk.MustExec("prepare s1 from 'select * from t where id = 1';") + tk.MustExec("prepare s2 from 'select * from t';") + tk.MustExec("set @@tidb_snapshot = " + ts) + tk.MustQuery("execute s1").Check(testkit.Rows("1 2")) + tk.MustQuery("execute s2").Check(testkit.Rows("1 2")) +} + +func TestPrepareCacheForPartition(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + + // TODO: if not already exists, include test with explicit partition selection + // both prunable and non-prunable + tk.MustExec("use test") + for _, pruneMode := range []string{string(variable.Static), string(variable.Dynamic)} { + planCacheUsed := "0" + if pruneMode == string(variable.Dynamic) { + planCacheUsed = "1" + } + tk.MustExec("set @@tidb_partition_prune_mode = '" + pruneMode + "'") + // Test for PointGet and IndexRead. + tk.MustExec("drop table if exists t_index_read") + tk.MustExec("create table t_index_read (id int, k int, c varchar(10), primary key (id, k)) partition by hash(id+k) partitions 10") + tk.MustExec("insert into t_index_read values (1, 2, 'abc'), (3, 4, 'def'), (5, 6, 'xyz')") + tk.MustExec("prepare stmt1 from 'select c from t_index_read where id = ? and k = ?;'") + tk.MustExec("set @id=1, @k=2") + // When executing one statement at the first time, we don't use cache, so we need to execute it at least twice to test the cache. + tk.MustQuery("execute stmt1 using @id, @k").Check(testkit.Rows("abc")) + tk.MustQuery("execute stmt1 using @id, @k").Check(testkit.Rows("abc")) + tk.MustExec("set @id=5, @k=6") + tk.MustQuery("execute stmt1 using @id, @k").Check(testkit.Rows("xyz")) + tk.MustExec("prepare stmt2 from 'select c from t_index_read where id = ? and k = ? and 1 = 1;'") + tk.MustExec("set @id=1, @k=2") + tk.MustQuery("execute stmt2 using @id, @k").Check(testkit.Rows("abc")) + tk.MustQuery("execute stmt2 using @id, @k").Check(testkit.Rows("abc")) + tk.MustExec("set @id=5, @k=6") + tk.MustQuery("execute stmt2 using @id, @k").Check(testkit.Rows("xyz")) + // Test for TableScan. + tk.MustExec("drop table if exists t_table_read") + tk.MustExec("create table t_table_read (id int, k int, c varchar(10), primary key(id)) partition by hash(id) partitions 10") + tk.MustExec("insert into t_table_read values (1, 2, 'abc'), (3, 4, 'def'), (5, 6, 'xyz')") + tk.MustExec("prepare stmt3 from 'select c from t_index_read where id = ?;'") + tk.MustExec("set @id=1") + // When executing one statement at the first time, we don't use cache, so we need to execute it at least twice to test the cache. + tk.MustQuery("execute stmt3 using @id").Check(testkit.Rows("abc")) + tk.MustQuery("execute stmt3 using @id").Check(testkit.Rows("abc")) + tk.MustExec("set @id=5") + tk.MustQuery("execute stmt3 using @id").Check(testkit.Rows("xyz")) + tk.MustExec("prepare stmt4 from 'select c from t_index_read where id = ? and k = ?'") + tk.MustExec("set @id=1, @k=2") + tk.MustQuery("execute stmt4 using @id, @k").Check(testkit.Rows("abc")) + tk.MustQuery("execute stmt4 using @id, @k").Check(testkit.Rows("abc")) + tk.MustExec("set @id=5, @k=6") + tk.MustQuery("execute stmt4 using @id, @k").Check(testkit.Rows("xyz")) + // Query on range partition tables should not raise error. + tk.MustExec("drop table if exists t_range_index") + tk.MustExec("create table t_range_index (id int, k int, c varchar(10), primary key(id)) partition by range(id) ( PARTITION p0 VALUES LESS THAN (4), PARTITION p1 VALUES LESS THAN (14),PARTITION p2 VALUES LESS THAN (20) )") + tk.MustExec("insert into t_range_index values (1, 2, 'abc'), (5, 4, 'def'), (13, 6, 'xyz'), (17, 6, 'hij')") + tk.MustExec(`analyze table t_range_index`) + tk.MustExec("prepare stmt5 from 'select c from t_range_index where id = ?'") + tk.MustExec("set @id=1") + tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("abc")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("abc")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=5") + tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("def")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("def")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=13") + tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("xyz")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=17") + tk.MustQuery("execute stmt5 using @id").Check(testkit.Rows("hij")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + + tk.MustExec("drop table if exists t_range_table") + tk.MustExec("create table t_range_table (id int, k int, c varchar(10)) partition by range(id) ( PARTITION p0 VALUES LESS THAN (4), PARTITION p1 VALUES LESS THAN (14),PARTITION p2 VALUES LESS THAN (20) )") + tk.MustExec("insert into t_range_table values (1, 2, 'abc'), (5, 4, 'def'), (13, 6, 'xyz'), (17, 6, 'hij')") + tk.MustExec(`analyze table t_range_table`) + tk.MustExec("prepare stmt6 from 'select c from t_range_table where id = ?'") + tk.MustExec("set @id=1") + tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("abc")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustExec(`set @@global.tidb_slow_log_threshold = -1`) + tk.MustQuery(`select * from information_schema.slow_query`).Check(testkit.Rows()) + // TODO: Check pruning, seems like it did not prune!!! + tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("abc")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=5") + tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("def")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("def")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=13") + tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("xyz")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=17") + tk.MustQuery("execute stmt6 using @id").Check(testkit.Rows("hij")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + + // Test for list partition + tk.MustExec("drop table if exists t_list_index") + tk.MustExec("create table t_list_index (id int, k int, c varchar(10), primary key(id)) partition by list (id*2-id) ( PARTITION p0 VALUES IN (1,2,3,4), PARTITION p1 VALUES IN (5,6,7,8),PARTITION p2 VALUES IN (9,10,11,12))") + tk.MustExec("insert into t_list_index values (1, 1, 'abc'), (5, 5, 'def'), (9, 9, 'xyz'), (12, 12, 'hij')") + tk.MustExec(`analyze table t_list_index`) + tk.MustExec("prepare stmt7 from 'select c from t_list_index where id = ?'") + tk.MustExec("set @id=1") + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("abc")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("abc")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=5") + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("def")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("def")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=9") + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("xyz")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=12") + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows("hij")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=100") + tk.MustQuery("execute stmt7 using @id").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + + // Test for list columns partition + tk.MustExec("drop table if exists t_list_index") + tk.MustExec("create table t_list_index (id int, k int, c varchar(10), primary key(id)) partition by list columns (id) ( PARTITION p0 VALUES IN (1,2,3,4), PARTITION p1 VALUES IN (5,6,7,8),PARTITION p2 VALUES IN (9,10,11,12))") + tk.MustExec("insert into t_list_index values (1, 1, 'abc'), (5, 5, 'def'), (9, 9, 'xyz'), (12, 12, 'hij')") + tk.MustExec(`analyze table t_list_index`) + tk.MustExec("prepare stmt8 from 'select c from t_list_index where id = ?'") + tk.MustExec("set @id=1") + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("abc")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("abc")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=5") + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("def")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("def")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=9") + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("xyz")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=12") + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows("hij")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustExec("set @id=100") + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + tk.MustQuery("execute stmt8 using @id").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(planCacheUsed)) + } +} + +func TestIssue33031(t *testing.T) { + store := testkit.CreateMockStore(t) + tkExplain := testkit.NewTestKit(t, store) + tkExplain.MustExec(`use test`) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk.MustExec(`use test`) + // https://github.com/pingcap/tidb/issues/33031 + tk.MustExec(`drop table if exists Issue33031`) + tk.MustExec(`CREATE TABLE Issue33031 (COL1 int(16) DEFAULT '29' COMMENT 'NUMERIC UNIQUE INDEX', COL2 bigint(20) DEFAULT NULL, UNIQUE KEY UK_COL1 (COL1)) PARTITION BY RANGE (COL1) (PARTITION P0 VALUES LESS THAN (0))`) + tk.MustExec(`insert into Issue33031 values(-5, 7)`) + tk.MustExec(`set @@session.tidb_partition_prune_mode='static'`) + tk.MustExec(`prepare stmt from 'select *,? from Issue33031 where col2 < ? and col1 in (?, ?)'`) + tk.MustExec(`set @a=111, @b=1, @c=2, @d=22`) + tk.MustQuery(`execute stmt using @d,@a,@b,@c`).Check(testkit.Rows()) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @a=112, @b=-2, @c=-5, @d=33`) + tk.MustQuery(`execute stmt using @d,@a,@b,@c`).Check(testkit.Rows("-5 7 33")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`deallocate prepare stmt`) + tk.MustExec(`set @@session.tidb_partition_prune_mode='dynamic'`) + tk.MustExec(`analyze table Issue33031`) + tk.MustExec(`prepare stmt from 'select *,? from Issue33031 where col2 < ? and col1 in (?, ?)'`) + tk.MustExec(`set @a=111, @b=1, @c=2, @d=22`) + tk.MustQuery(`execute stmt using @d,@a,@b,@c`).Check(testkit.Rows()) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + // TODO: Add another test that will use the plan cache instead of [Batch]PointGet + // when supported for partitioned tables as well. + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustExec(`set @a=112, @b=-2, @c=-5, @d=33`) + tk.MustQuery(`execute stmt using @d,@a,@b,@c`).Check(testkit.Rows("-5 7 33")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + explain := tkExplain.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Regexp(t, "IndexLookUp", explain[1][0]) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustExec(`alter table Issue33031 remove partitioning`) + tk.MustQuery(`execute stmt using @d,@a,@b,@c`).Check(testkit.Rows("-5 7 33")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: Batch/PointGet plans may be over-optimized")) + tk.MustQuery(`execute stmt using @d,@a,@b,@c`).Check(testkit.Rows("-5 7 33")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + explain = tkExplain.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Regexp(t, "Batch_Point_Get", explain[2][0]) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip prepared plan-cache: Batch/PointGet plans may be over-optimized")) +} + +func newSession(t *testing.T, store kv.Storage, dbName string) sessiontypes.Session { + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + mustExec(t, se, "create database if not exists "+dbName) + mustExec(t, se, "use "+dbName) + return se +} + +func mustExec(t *testing.T, se sessiontypes.Session, sql string) { + _, err := se.Execute(context.Background(), sql) + require.NoError(t, err) +} + +func TestPlanCacheUnionScan(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`) // insert-tmt can hit the cache and affect hit counter in this UT + pb := &dto.Metric{} + metrics.ResettablePlanCacheCounterFortTest = true + metrics.PlanCacheCounter.Reset() + counter := metrics.PlanCacheCounter.WithLabelValues("prepare") + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1(a int not null)") + tk.MustExec("create table t2(a int not null)") + tk.MustExec("prepare stmt1 from 'select * from t1 where a > ?'") + tk.MustExec("set @p0 = 0") + tk.MustQuery("execute stmt1 using @p0").Check(testkit.Rows()) + tk.MustExec("begin") + tk.MustQuery("execute stmt1 using @p0").Check(testkit.Rows()) + cnt := pb.GetCounter().GetValue() + require.Equal(t, float64(0), cnt) // can't reuse the plan created outside the txn + tk.MustQuery("execute stmt1 using @p0").Check(testkit.Rows()) + err := counter.Write(pb) + require.NoError(t, err) + require.Equal(t, float64(0), cnt) + tk.MustExec("insert into t1 values(1)") + // Cached plan is invalid now, it is not chosen and removed. + tk.MustQuery("execute stmt1 using @p0").Check(testkit.Rows( + "1", + )) + err = counter.Write(pb) + require.NoError(t, err) + cnt = pb.GetCounter().GetValue() + require.Equal(t, float64(1), cnt) + tk.MustExec("insert into t2 values(1)") + // Cached plan is chosen, modification on t2 does not impact plan of t1. + tk.MustQuery("execute stmt1 using @p0").Check(testkit.Rows( + "1", + )) + err = counter.Write(pb) + require.NoError(t, err) + cnt = pb.GetCounter().GetValue() + require.Equal(t, float64(2), cnt) + tk.MustExec("rollback") + // Though cached plan contains UnionScan, it does not impact correctness, so it is reused. + tk.MustQuery("execute stmt1 using @p0").Check(testkit.Rows()) + err = counter.Write(pb) + require.NoError(t, err) + cnt = pb.GetCounter().GetValue() + require.Equal(t, float64(3), cnt) + + tk.MustExec("prepare stmt2 from 'select * from t1 left join t2 on true where t1.a > ?'") + tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows()) + tk.MustExec("begin") + tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows()) + require.Equal(t, float64(3), cnt) // can't reuse the plan created outside the txn + tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows()) + err = counter.Write(pb) + require.NoError(t, err) + cnt = pb.GetCounter().GetValue() + require.Equal(t, float64(4), cnt) + tk.MustExec("insert into t1 values(1)") + // Cached plan is invalid now, it is not chosen and removed. + tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows( + "1 ", + )) + err = counter.Write(pb) + require.NoError(t, err) + cnt = pb.GetCounter().GetValue() + require.Equal(t, float64(4), cnt) + tk.MustExec("insert into t2 values(1)") + // Cached plan is invalid now, it is not chosen and removed. + tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows( + "1 1", + )) + err = counter.Write(pb) + require.NoError(t, err) + cnt = pb.GetCounter().GetValue() + require.Equal(t, float64(4), cnt) + // Cached plan is reused. + tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows( + "1 1", + )) + err = counter.Write(pb) + require.NoError(t, err) + cnt = pb.GetCounter().GetValue() + require.Equal(t, float64(5), cnt) + tk.MustExec("rollback") + // Though cached plan contains UnionScan, it does not impact correctness, so it is reused. + tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows()) + err = counter.Write(pb) + require.NoError(t, err) + cnt = pb.GetCounter().GetValue() + require.Equal(t, float64(6), cnt) +} + +func TestPlanCacheSwitchDB(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + + // create a table in test + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists t`) + tk.MustExec(`create table t(a int)`) + tk.MustExec(`insert into t values (-1)`) + tk.MustExec(`prepare stmt from 'select * from t'`) + + // DB is not specified + se2, err := session.CreateSession4TestWithOpt(store, &session.Opt{ + PreparedPlanCache: core.NewLRUPlanCache(100, 0.1, math.MaxUint64, tk.Session(), false), + }) + require.NoError(t, err) + tk2 := testkit.NewTestKitWithSession(t, store, se2) + require.Equal(t, tk2.ExecToErr(`prepare stmt from 'select * from t'`).Error(), "[planner:1046]No database selected") + require.Equal(t, tk2.ExecToErr(`prepare stmt from 'select * from test.t'`), nil) + + // switch to a new DB + tk.MustExec(`drop database if exists plan_cache`) + tk.MustExec(`create database plan_cache`) + tk.MustExec(`use plan_cache`) + tk.MustExec(`create table t(a int)`) + tk.MustExec(`insert into t values (1)`) + tk.MustQuery(`execute stmt`).Check(testkit.Rows("-1")) // read test.t + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustQuery(`execute stmt`).Check(testkit.Rows("-1")) // read test.t + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + // prepare again + tk.MustExec(`prepare stmt from 'select * from t'`) + tk.MustQuery(`execute stmt`).Check(testkit.Rows("1")) // read plan_cache.t + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustQuery(`execute stmt`).Check(testkit.Rows("1")) // read plan_cache.t + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + + // specify DB in the query + tk.MustExec(`prepare stmt from 'select * from test.t'`) + tk.MustQuery(`execute stmt`).Check(testkit.Rows("-1")) // read test.t + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustQuery(`execute stmt`).Check(testkit.Rows("-1")) // read test.t + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) +} + +func TestInvisibleIndexPrepare(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, unique idx_a(a))") + tk.MustExec("insert into t values(1)") + tk.MustExec(`prepare stmt1 from "select a from t order by a"`) + + tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.IndexNames, 1) + require.Equal(t, "t:idx_a", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) + + tk.MustExec("alter table t alter index idx_a invisible") + tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.IndexNames, 0) + + tk.MustExec("alter table t alter index idx_a visible") + tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.IndexNames, 1) + require.Equal(t, "t:idx_a", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) +} + +func TestPlanCacheSnapshot(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int)") + tk.MustExec("insert into t values (1),(2),(3),(4)") + + // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. + timeSafe := time.Now().Add(-48 * 60 * 60 * time.Second).Format("20060102-15:04:05 -0700 MST") + safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') + ON DUPLICATE KEY + UPDATE variable_value = '%[1]s'` + tk.MustExec(fmt.Sprintf(safePointSQL, timeSafe)) + + tk.MustExec("prepare stmt from 'select * from t where id=?'") + tk.MustExec("set @p = 1") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt using @p").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt using @p").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + + // Record the current tso. + tk.MustExec("begin") + tso := tk.Session().GetSessionVars().TxnCtx.StartTS + tk.MustExec("rollback") + require.True(t, tso > 0) + // Insert one more row with id = 1. + tk.MustExec("insert into t values (1)") + + tk.MustExec(fmt.Sprintf("set @@tidb_snapshot = '%d'", tso)) + tk.MustQuery("select * from t where id = 1").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt using @p").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) +} + +func TestPartitionTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + + // enable partition table dynamic mode + tk.MustExec("create database test_plan_cache") + tk.MustExec("use test_plan_cache") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("set @@tidb_enable_list_partition = 1") + + seed := time.Now().UnixNano() + //seed := int64(1704191012078910000) + t.Logf("seed: %d", seed) + randomSeed := rand.New(rand.NewSource(seed)) + + type testcase struct { + t1Create string + t2Create string + rowGener func() string + varGener func() string + query string + } + randDateTime := func() string { + return fmt.Sprintf("%v-%v-%v %v:%v:%v", + 1950+randomSeed.Intn(100), 1+randomSeed.Intn(12), 1+randomSeed.Intn(28), // date + randomSeed.Intn(24), randomSeed.Intn(60), randomSeed.Intn(60)) // time + } + randDate := func() string { + return fmt.Sprintf("%v-%v-%v", 1950+randomSeed.Intn(100), 1+randomSeed.Intn(12), 1+randomSeed.Intn(28)) + } + testcases := []testcase{ + { // hash partition + int + "create table t1(a int, b int) partition by hash(a) partitions 20", + "create table t2(a int, b int)", + func() string { return fmt.Sprintf("(%v, %v)", randomSeed.Intn(100000000), randomSeed.Intn(100000000)) }, + func() string { return fmt.Sprintf("%v", randomSeed.Intn(100000000)) }, + `select * from %v where a > ?`, + }, + { // range partition + int + `create table t1(a int, b int) partition by range(a) ( + partition p0 values less than (20000000), + partition p1 values less than (40000000), + partition p2 values less than (60000000), + partition p3 values less than (80000000), + partition p4 values less than (100000000))`, + `create table t2(a int, b int)`, + func() string { return fmt.Sprintf("(%v, %v)", randomSeed.Intn(100000000), randomSeed.Intn(100000000)) }, + func() string { return fmt.Sprintf("%v", randomSeed.Intn(100000000)) }, + `select * from %v where a > ?`, + }, + { // range partition + varchar + `create table t1(a varchar(10), b varchar(10)) partition by range columns(a) ( + partition p0 values less than ('200'), + partition p1 values less than ('400'), + partition p2 values less than ('600'), + partition p3 values less than ('800'), + partition p4 values less than ('9999'))`, + `create table t2(a varchar(10), b varchar(10))`, + func() string { return fmt.Sprintf(`("%v", "%v")`, randomSeed.Intn(1000), randomSeed.Intn(1000)) }, + func() string { return fmt.Sprintf(`"%v"`, randomSeed.Intn(1000)) }, + `select * from %v where a > ?`, + }, + { // range partition + datetime + `create table t1(a datetime, b datetime) partition by range columns(a) ( + partition p0 values less than ('1970-01-01 00:00:00'), + partition p1 values less than ('1990-01-01 00:00:00'), + partition p2 values less than ('2010-01-01 00:00:00'), + partition p3 values less than ('2030-01-01 00:00:00'), + partition p4 values less than ('2060-01-01 00:00:00'))`, + `create table t2(a datetime, b datetime)`, + func() string { return fmt.Sprintf(`("%v", "%v")`, randDateTime(), randDateTime()) }, + func() string { return fmt.Sprintf(`"%v"`, randDateTime()) }, + `select * from %v where a > ?`, + }, + { // range partition + date + `create table t1(a date, b date) partition by range columns(a) ( + partition p0 values less than ('1970-01-01'), + partition p1 values less than ('1990-01-01'), + partition p2 values less than ('2010-01-01'), + partition p3 values less than ('2030-01-01'), + partition p4 values less than ('2060-01-01'))`, + `create table t2(a date, b date)`, + func() string { return fmt.Sprintf(`("%v", "%v")`, randDate(), randDate()) }, + func() string { return fmt.Sprintf(`"%v"`, randDate()) }, + `select * from %v where a > ?`, + }, + { // list partition + int + // This should be able to use plan cache if @a between 14-18 + `create table t1(a int, b int) partition by list(a) ( + partition p0 values in (0, 1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8, 9), + partition p2 values in (10, 11, 12, 13, 14), + partition p3 values in (15, 16, 17, 18, 19))`, + `create table t2(a int, b int)`, + func() string { return fmt.Sprintf("(%v, %v)", randomSeed.Intn(20), randomSeed.Intn(20)) }, + func() string { return fmt.Sprintf("%v", 0) }, + //func() string { return fmt.Sprintf("%v", randomSeed.Intn(20)) }, + `select * from %v where a > ?`, + }, + } + for _, tc := range testcases { + // create tables and insert some records + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec(tc.t1Create) + tk.MustExec(tc.t2Create) + vals := make([]string, 0, 2048) + for i := 0; i < 2048; i++ { + vals = append(vals, tc.rowGener()) + } + tk.MustExec(fmt.Sprintf("insert into t1 values %s", strings.Join(vals, ","))) + tk.MustExec(fmt.Sprintf("insert into t2 values %s", strings.Join(vals, ","))) + // if global stats are not ready, then dynamic prune mode is not used! + // TODO: Test without these to prevent prepared plan cache to switch between dynamic + // and static prune mode. + tk.MustExec(`analyze table t1`) + tk.MustExec(`analyze table t2`) + + // the first query, @last_plan_from_cache should be zero + tk.MustExec(fmt.Sprintf(`prepare stmt1 from "%s"`, fmt.Sprintf(tc.query, "t1"))) + tk.MustExec(fmt.Sprintf(`prepare stmt2 from "%s"`, fmt.Sprintf(tc.query, "t2"))) + tk.MustExec(fmt.Sprintf("set @a=%v", tc.varGener())) + result1 := tk.MustQuery("execute stmt1 using @a").Sort().Rows() + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt2 using @a").Sort().Check(result1) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + commentString := fmt.Sprintf("/*\ntc.query=%s\ntc.t1Create=%s */", tc.query, tc.t1Create) + numWarns := 0 + for i := 0; i < 100; i++ { + val := tc.varGener() + t.Logf("@a=%v", val) + tk.MustExec(fmt.Sprintf("set @a=%v", val)) + result1 = tk.MustQuery("execute stmt1 using @a /* @a=" + val + " i=" + strconv.Itoa(i) + " */ " + commentString).Sort().Rows() + foundInPlanCache := tk.Session().GetSessionVars().FoundInPlanCache + warnings := tk.MustQuery(`show warnings`) + if len(warnings.Rows()) > 0 { + warnings.CheckContain("skip plan-cache: plan rebuild failed, ") + numWarns++ + } else { + require.True(t, foundInPlanCache, "select @@last_plan_from_cache /* i=%d prepared statement: (t1) %s\n-- create table: %s*/", i, tc.query, tc.t1Create) + } + tk.MustQuery("execute stmt2 using @a /* @a=" + val + " i=" + strconv.Itoa(i) + " */ " + commentString).Sort().Check(result1) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + } + require.Less(t, numWarns, 5, "Too many skip plan-cache warnings!") + t.Logf("Create t1: %s\nstmt: %s\nnumWarns: %d", tc.t1Create, tc.query, numWarns) + } +} + +func helperCheckPlanCache(t *testing.T, tk *testkit.TestKit, sql, expected string, arr []string) []string { + res := tk.MustQuery(sql) + got := res.Rows()[0][0] + if expected == "0" { + require.Equal(t, expected, got, sql) + } else { + if got != expected { + return append(arr, sql) + } + } + return arr +} + +func TestPartitionWithVariedDataSources(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + + // enable partition table dynamic mode + tk.MustExec("create database test_plan_cache2") + tk.MustExec("use test_plan_cache2") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // prepare tables + tk.MustExec(`create table trangePK (a int primary key, b int) partition by range (a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + tk.MustExec(`create table thashPK (a int primary key, b int) partition by hash (a) partitions 4`) + tk.MustExec(`create table tnormalPK (a int primary key, b int)`) + tk.MustExec(`create table trangeIdx (a int unique key, b int) partition by range (a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + tk.MustExec(`create table thashIdx (a int unique key, b int) partition by hash (a) partitions 4`) + tk.MustExec(`create table tnormalIdx (a int unique key, b int)`) + uniqueVals := make(map[int]struct{}) + vals := make([]string, 0, 1000) + for len(vals) < 1000 { + a := rand.Intn(40000) + if _, ok := uniqueVals[a]; ok { + continue + } + uniqueVals[a] = struct{}{} + b := rand.Intn(40000) + vals = append(vals, fmt.Sprintf("(%v, %v)", a, b)) + } + for _, tbl := range []string{"trangePK", "thashPK", "tnormalPK", "trangeIdx", "thashIdx", "tnormalIdx"} { + tk.MustExec(fmt.Sprintf(`insert into %v values %v`, tbl, strings.Join(vals, ", "))) + tk.MustExec(`analyze table ` + tbl) + } + + missedPlanCache := make([]string, 0, 4) + // TableReader, PointGet on PK, BatchGet on PK + for _, tbl := range []string{`trangePK`, `thashPK`, `tnormalPK`} { + tk.MustExec(fmt.Sprintf(`prepare stmt%v_tablescan from 'select * from %v use index(primary) where a > ? and a < ?'`, tbl, tbl)) + tk.MustExec(fmt.Sprintf(`prepare stmt%v_pointget from 'select * from %v use index(primary) where a = ?'`, tbl, tbl)) + tk.MustExec(fmt.Sprintf(`prepare stmt%v_batchget from 'select * from %v use index(primary) where a in (?, ?, ?)'`, tbl, tbl)) + } + loops := 100 + for i := 0; i < loops; i++ { + mina, maxa := rand.Intn(40000), rand.Intn(40000) + pointa := mina + // Allow out-of-range to trigger edge cases for non-matching partition + a0, a1, a2 := mina, maxa, rand.Intn(41000) + if i == 1 { + // test what happens if we have duplicates + a2 = a0 + } + if mina > maxa { + mina, maxa = maxa, mina + } + tk.MustExec(fmt.Sprintf(`set @mina=%v, @maxa=%v`, mina, maxa)) + tk.MustExec(fmt.Sprintf(`set @pointa=%v`, pointa)) + tk.MustExec(fmt.Sprintf(`set @a0=%v, @a1=%v, @a2=%v`, a0, a1, a2)) + + var rscan, rpoint, rbatch [][]any + for id, tbl := range []string{`trangePK`, `thashPK`, `tnormalPK`} { + scan := tk.MustQuery(fmt.Sprintf(`execute stmt%v_tablescan using @mina, @maxa`, tbl)).Sort() + if id == 0 { + rscan = scan.Rows() + } else { + scan.Check(rscan) + } + if i > 0 { + tblStr := ` table: ` + tbl + " i :" + strconv.FormatInt(int64(i), 10) + " */" + missedPlanCache = helperCheckPlanCache(t, tk, `select @@last_plan_from_cache /* tablescan table: `+tblStr, "1", missedPlanCache) + } + + point := tk.MustQuery(fmt.Sprintf(`execute stmt%v_pointget using @pointa`, tbl)).Sort() + if id == 0 { + rpoint = point.Rows() + } else { + point.Check(rpoint) + } + if i > 0 { + tblStr := ` table: ` + tbl + " i :" + strconv.FormatInt(int64(i), 10) + " */" + missedPlanCache = helperCheckPlanCache(t, tk, `select @@last_plan_from_cache /* pointget table: `+tblStr, "1", missedPlanCache) + } + + batch := tk.MustQuery(fmt.Sprintf(`execute stmt%v_batchget using @a0, @a1, @a2`, tbl)).Sort() + if id == 0 { + rbatch = batch.Rows() + } else { + batch.Check(rbatch) + } + if i > 0 { + tblStr := ` table: ` + tbl + " i :" + strconv.FormatInt(int64(i), 10) + " */" + missedPlanCache = helperCheckPlanCache(t, tk, `select @@last_plan_from_cache /* batchget table: `+tblStr, "1", missedPlanCache) + } + } + } + + // IndexReader, IndexLookUp, PointGet on Idx, BatchGet on Idx + for _, tbl := range []string{"trangeIdx", "thashIdx", "tnormalIdx"} { + tk.MustExec(fmt.Sprintf(`prepare stmt%v_indexscan from 'select a from %v use index(a) where a > ? and a < ?'`, tbl, tbl)) + tk.MustExec(fmt.Sprintf(`prepare stmt%v_indexlookup from 'select * from %v use index(a) where a > ? and a < ?'`, tbl, tbl)) + tk.MustExec(fmt.Sprintf(`prepare stmt%v_pointget_idx from 'select * from %v use index(a) where a = ?'`, tbl, tbl)) + tk.MustExec(fmt.Sprintf(`prepare stmt%v_batchget_idx from 'select * from %v use index(a) where a in (?, ?, ?)'`, tbl, tbl)) + } + for i := 0; i < loops; i++ { + mina, maxa := rand.Intn(40000), rand.Intn(40000) + if mina > maxa { + mina, maxa = maxa, mina + } + tk.MustExec(fmt.Sprintf(`set @mina=%v, @maxa=%v`, mina, maxa)) + tk.MustExec(fmt.Sprintf(`set @pointa=%v`, rand.Intn(40000))) + // Allow out-of-range to trigger edge cases for non-matching partition + a0, a1, a2 := rand.Intn(40000), rand.Intn(40000), rand.Intn(41000) + if i == 1 { + // test what happens if we have duplicates + a2 = a0 + } + tk.MustExec(fmt.Sprintf(`set @a0=%v, @a1=%v, @a2=%v`, a0, a1, a2)) + + var rscan, rlookup, rpoint, rbatch [][]any + for id, tbl := range []string{"trangeIdx", "thashIdx", "tnormalIdx"} { + scan := tk.MustQuery(fmt.Sprintf(`execute stmt%v_indexscan using @mina, @maxa`, tbl)).Sort() + tblStr := ` table: ` + tbl + " i :" + strconv.FormatInt(int64(i), 10) + " */" + if i > 0 { + missedPlanCache = helperCheckPlanCache(t, tk, `select @@last_plan_from_cache /* indexscan table: `+tblStr, "1", missedPlanCache) + } + if id == 0 { + rscan = scan.Rows() + } else { + scan.Check(rscan) + } + + lookup := tk.MustQuery(fmt.Sprintf(`execute stmt%v_indexlookup using @mina, @maxa`, tbl)).Sort() + if i > 0 { + missedPlanCache = helperCheckPlanCache(t, tk, `select @@last_plan_from_cache /* indexlookup table: `+tblStr, "1", missedPlanCache) + } + if id == 0 { + rlookup = lookup.Rows() + } else { + lookup.Check(rlookup) + } + + point := tk.MustQuery(fmt.Sprintf(`execute stmt%v_pointget_idx using @pointa`, tbl)).Sort() + if tbl == `tnormalPK` && i > 0 { + // PlanCache cannot support PointGet now since we haven't relocated partition after rebuilding range. + // Please see Execute.rebuildRange for more details. + missedPlanCache = helperCheckPlanCache(t, tk, `select @@last_plan_from_cache /* pointget table: `+tblStr, "1", missedPlanCache) + } + if id == 0 { + rpoint = point.Rows() + } else { + point.Check(rpoint) + } + + batch := tk.MustQuery(fmt.Sprintf(`execute stmt%v_batchget_idx using @a0, @a1, @a2`, tbl)).Sort() + if i > 0 { + missedPlanCache = helperCheckPlanCache(t, tk, `select @@last_plan_from_cache /* batchget table: `+tblStr, "1", missedPlanCache) + } + if id == 0 { + rbatch = batch.Rows() + } else { + batch.Check(rbatch) + } + } + } + // Allow <=10% non-cached queries, due to background changes etc. + // + 3 intentionally misses, due to duplicate values + // Notice there are 3 tables * 4 queries per loop :) + if len(missedPlanCache) > (3 + 10*loops*3*4/100) { + require.Equal(t, []string{}, missedPlanCache) + } +} + +func TestCachedTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + + tk.MustExec("create table t (a int, b int, index i_b(b))") + tk.MustExec("insert into t values (1, 1), (2, 2)") + tk.MustExec("alter table t cache") + + tk.MustExec("prepare tableScan from 'select * from t where a>=?'") + tk.MustExec("prepare indexScan from 'select b from t use index(i_b) where b>?'") + tk.MustExec("prepare indexLookup from 'select a from t use index(i_b) where b>? and b1 and b<3 --> b=2 + + // PointGet + tk.MustQuery("execute pointGet using @a").Check(testkit.Rows("1")) + require.True(t, lastReadFromCache(tk)) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) +} + +func TestPlanCacheWithRCWhenInfoSchemaChange(t *testing.T) { + ctx := context.Background() + store := testkit.CreateMockStore(t) + + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk1.MustExec("set global tidb_enable_metadata_lock=0") + tk2.MustExec("use test") + tk1.MustExec("drop table if exists t1") + tk1.MustExec("create table t1(id int primary key, c int, index ic (c))") + // prepare text protocol + tk1.MustExec("prepare s from 'select /*+use_index(t1, ic)*/ * from t1 where 1'") + // prepare binary protocol + stmtID, _, _, err := tk2.Session().PrepareStmt("select /*+use_index(t1, ic)*/ * from t1 where 1") + require.Nil(t, err) + tk1.MustExec("set tx_isolation='READ-COMMITTED'") + tk1.MustExec("begin pessimistic") + tk2.MustExec("set tx_isolation='READ-COMMITTED'") + tk2.MustExec("begin pessimistic") + tk1.MustQuery("execute s").Check(testkit.Rows()) + rs, err := tk2.Session().ExecutePreparedStmt(ctx, stmtID, expression.Args2Expressions4Test()) + require.Nil(t, err) + tk2.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows()) + + tk3 := testkit.NewTestKit(t, store) + tk3.MustExec("use test") + tk3.MustExec("alter table t1 drop index ic") + tk3.MustExec("insert into t1 values(1, 0)") + + // The execution after schema changed should not hit plan cache. + // execute text protocol + tk1.MustQuery("execute s").Check(testkit.Rows("1 0")) + tk1.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + // execute binary protocol + rs, err = tk2.Session().ExecutePreparedStmt(ctx, stmtID, expression.Args2Expressions4Test()) + require.Nil(t, err) + tk2.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 0")) + tk2.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) +} + +func TestConsistencyBetweenPrepareExecuteAndNormalSql(t *testing.T) { + ctx := context.Background() + store := testkit.CreateMockStore(t) + + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk1.MustExec("set global tidb_enable_metadata_lock=0") + tk1.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk2.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk1.MustExec("use test") + tk2.MustExec("use test") + tk1.MustExec("drop table if exists t1") + tk1.MustExec("create table t1(id int primary key, c int)") + tk1.MustExec("insert into t1 values(1, 1), (2, 2)") + // prepare text protocol + tk1.MustExec("prepare s from 'select * from t1'") + // prepare binary protocol + stmtID, _, _, err := tk1.Session().PrepareStmt("select * from t1") + require.Nil(t, err) + tk1.MustExec("set tx_isolation='READ-COMMITTED'") + tk1.MustExec("begin pessimistic") + tk2.MustExec("set tx_isolation='READ-COMMITTED'") + tk2.MustExec("begin pessimistic") + + // Execute using sql + tk1.MustQuery("execute s").Check(testkit.Rows("1 1", "2 2")) + // Execute using binary + rs, err := tk1.Session().ExecutePreparedStmt(ctx, stmtID, expression.Args2Expressions4Test()) + require.Nil(t, err) + tk1.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 1", "2 2")) + // Normal sql + tk1.MustQuery("select * from t1").Check(testkit.Rows("1 1", "2 2")) + + // Change infoSchema + tk2.MustExec("alter table t1 drop column c") + tk2.MustExec("insert into t1 values (3)") + // Execute using sql + tk1.MustQuery("execute s").Check(testkit.Rows("1 1", "2 2", "3 ")) + // Execute using binary + rs, err = tk1.Session().ExecutePreparedStmt(ctx, stmtID, expression.Args2Expressions4Test()) + require.Nil(t, err) + tk1.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 1", "2 2", "3 ")) + // Normal sql + tk1.MustQuery("select * from t1").Check(testkit.Rows("1 1", "2 2", "3 ")) + tk1.MustExec("commit") + + // After beginning a new txn, the infoSchema should be the latest + tk1.MustExec("begin pessimistic") + tk1.MustQuery("select * from t1").Check(testkit.Rows("1", "2", "3")) +} + +func verifyCache(ctx context.Context, t *testing.T, tk1 *testkit.TestKit, tk2 *testkit.TestKit, stmtID uint32) { + // Cache miss in the firs time. + tk1.MustExec("execute s") + tk1.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + // This time, the cache will be hit. + rs, err := tk1.Session().ExecutePreparedStmt(ctx, stmtID, expression.Args2Expressions4Test()) + require.NoError(t, err) + require.NoError(t, rs.Close()) + tk1.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk1.MustExec("execute s") + tk1.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + + // Change infoSchema version which will make the plan cache invalid in the next execute + // DDL is blocked by MDL. + //tk2.MustExec("alter table t1 drop column c") + //tk1.MustExec("execute s") + //tk1.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + //// Now the plan cache will be valid + //rs, err = tk1.Session().ExecutePreparedStmt(ctx, stmtID, expression.Args2Expressions4Test()) + //require.NoError(t, err) + //require.NoError(t, rs.Close()) + //tk1.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) +} + +func TestCacheHitInRc(t *testing.T) { + ctx := context.Background() + store := testkit.CreateMockStore(t) + + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk1.MustExec("set global tidb_enable_metadata_lock=0") + tk1.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk2.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk1.MustExec("use test") + tk2.MustExec("use test") + tk1.MustExec("drop table if exists t1") + tk1.MustExec("create table t1(id int primary key, c int)") + tk1.MustExec("insert into t1 values(1, 1), (2, 2)") + // prepare text protocol + tk1.MustExec("prepare s from 'select * from t1'") + // prepare binary protocol + stmtID, _, _, err := tk1.Session().PrepareStmt("select * from t1") + require.Nil(t, err) + + // Test for RC + tk1.MustExec("set tx_isolation='READ-COMMITTED'") + tk1.MustExec("begin pessimistic") + + // Verify for the RC isolation + verifyCache(ctx, t, tk1, tk2, stmtID) + tk1.MustExec("rollback") +} + +func TestCacheHitInForUpdateRead(t *testing.T) { + ctx := context.Background() + store := testkit.CreateMockStore(t) + + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk1.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk2.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk1.MustExec("use test") + tk2.MustExec("use test") + tk1.MustExec("drop table if exists t1") + tk1.MustExec("create table t1(id int primary key, c int)") + tk1.MustExec("insert into t1 values(1, 1), (2, 2)") + + tk1.MustExec("prepare s from 'select * from t1 where id = 1 for update'") + stmtID, _, _, err := tk1.Session().PrepareStmt("select * from t1 where id = 1 for update") + require.Nil(t, err) + tk1.MustExec("begin pessimistic") + + // Verify for the for update read + verifyCache(ctx, t, tk1, tk2, stmtID) + tk1.MustExec("rollback") +} + +func TestPointGetForUpdateAutoCommitCache(t *testing.T) { + ctx := context.Background() + store := testkit.CreateMockStore(t) + + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk1.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk2.MustExec(`set tidb_enable_prepared_plan_cache=1`) + tk1.MustExec("use test") + tk2.MustExec("use test") + tk1.MustExec("drop table if exists t1") + tk1.MustExec("create table t1(id int primary key, c int)") + tk1.MustExec("insert into t1 values(1, 1), (2, 2)") + + tk1.MustExec("prepare s from 'select * from t1 where id = 1 for update'") + stmtID, _, _, err := tk1.Session().PrepareStmt("select * from t1 where id = 1 for update") + require.Nil(t, err) + rs, err := tk1.Session().ExecutePreparedStmt(ctx, stmtID, expression.Args2Expressions4Test()) + require.Nil(t, err) + tk1.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 1")) + tk1.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + rs, err = tk1.Session().ExecutePreparedStmt(ctx, stmtID, expression.Args2Expressions4Test()) + require.Nil(t, err) + tk1.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 1")) + tk1.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + + tk2.MustExec("alter table t1 drop column c") + tk2.MustExec("update t1 set id = 10 where id = 1") + + rs, err = tk1.Session().ExecutePreparedStmt(ctx, stmtID, expression.Args2Expressions4Test()) + require.Nil(t, err) + tk1.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows()) + tk1.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + rs, err = tk1.Session().ExecutePreparedStmt(ctx, stmtID, expression.Args2Expressions4Test()) + require.Nil(t, err) + tk1.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows()) + tk1.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) +} + +func TestPrepareCacheForDynamicPartitionPruning(t *testing.T) { + // https://github.com/pingcap/tidb/issues/33031 + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustQuery(`select @@session.tidb_enable_prepared_plan_cache`).Check(testkit.Rows("1")) + + tk.MustExec("use test") + tkExplain := testkit.NewTestKit(t, store) + tkExplain.MustExec("use test") + for _, pruneMode := range []string{string(variable.Static), string(variable.Dynamic)} { + tk.MustExec("set @@tidb_partition_prune_mode = '" + pruneMode + "'") + + tk.MustExec(`drop table if exists t`) + tk.MustExec(`CREATE TABLE t (a int(16), b bigint, UNIQUE KEY (a)) PARTITION BY RANGE (a) (PARTITION P0 VALUES LESS THAN (0))`) + tk.MustExec(`insert into t values(-5, 7)`) + tk.MustExec(`analyze table t`) + tk.MustExec(`prepare stmt from 'select * from t where a = ? and b < ?'`) + tk.MustExec(`set @a=1, @b=111`) + // Note that this is not matching any partition! + tk.MustQuery(`execute stmt using @a,@b`).Check(testkit.Rows()) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + explain := tkExplain.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + if pruneMode == string(variable.Dynamic) { + require.Equal(t, "Selection_6", explain.Rows()[0][0]) + } else { + require.Equal(t, "TableDual_7", explain.Rows()[0][0]) + } + tk.MustExec(`set @a=-5, @b=112`) + tk.MustQuery(`execute stmt using @a,@b`).Check(testkit.Rows("-5 7")) + + explain = tkExplain.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + if pruneMode == string(variable.Dynamic) { + require.Equal(t, "Selection_6", explain.Rows()[0][0]) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + } else { + explain.CheckAt([]int{0}, + [][]any{ + {"Selection_8"}, + {"└─Point_Get_7"}, + }) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1105 skip prepared plan-cache: query accesses partitioned tables is un-cacheable if tidb_partition_pruning_mode = 'static'")) + } + + // Test TableDual + tk.MustExec(`set @b=5, @a=113`) + tk.MustQuery(`execute stmt using @a,@b`).Check(testkit.Rows()) + require.Equal(t, pruneMode == string(variable.Dynamic), tk.Session().GetSessionVars().FoundInPlanCache) + } +} + +func TestHashPartitionAndPlanCache(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustQuery(`select @@session.tidb_enable_prepared_plan_cache`).Check(testkit.Rows("1")) + tkExplain := testkit.NewTestKit(t, store) + tkExplain.MustExec("use test") + + tk.MustExec("use test") + tk.MustExec(`drop table if exists t`) + tk.MustExec(`CREATE TABLE t (b varchar(255), a int primary key nonclustered, key (b)) PARTITION BY HASH (a) partitions 5`) + tk.MustExec(`insert into t values(0,0),(1,1),(2,2),(3,3),(4,4)`) + tk.MustExec(`insert into t select b + 5, a + 5 from t`) + tk.MustExec(`analyze table t`) + + // Point get PK + tk.MustExec(`prepare stmt from 'select * from t where a = ?'`) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustExec(`set @a=1`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("1 1")) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + explain := tkExplain.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + require.Equal(t, "Point_Get_1", explain.Rows()[0][0]) + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + + tk.MustExec(`set @a=2`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("2 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + explain = tkExplain.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + require.Equal(t, "Point_Get_1", explain.Rows()[0][0]) + tk.MustExec(`drop table t`) + + tk.MustExec(`CREATE TABLE t (b varchar(255), a int, key (b), unique key (a)) PARTITION BY HASH (a) partitions 5`) + tk.MustExec(`insert into t values(0,0),(1,1),(2,2),(3,3),(4,4)`) + tk.MustExec(`insert into t select b + 5, a + 5 from t`) + tk.MustExec(`analyze table t`) + + // Point get Unique Key + tk.MustExec(`prepare stmt from 'select * from t where a = ?'`) + tk.MustExec(`set @a=1`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("1 1")) + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + tk.MustExec(`set @a=2`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows("2 2")) + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + explain = tkExplain.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + require.Equal(t, "Point_Get_1", explain.Rows()[0][0]) +}